diff --git a/pkg/bootstrap/versions/v2_0_2/cluster_upgrade_list.go b/pkg/bootstrap/versions/v2_0_2/cluster_upgrade_list.go index 6e6d0fdc1b3b8..3e3b5123206ae 100644 --- a/pkg/bootstrap/versions/v2_0_2/cluster_upgrade_list.go +++ b/pkg/bootstrap/versions/v2_0_2/cluster_upgrade_list.go @@ -22,6 +22,9 @@ import ( var clusterUpgEntries = []versions.UpgradeEntry{ upg_mo_cdc_watermark, + upg_mo_pubs_add_account_name, + upg_mo_subs_add_sub_account_name, + upg_mo_subs_add_pub_account_id, } var upg_mo_cdc_watermark = versions.UpgradeEntry{ @@ -37,3 +40,48 @@ var upg_mo_cdc_watermark = versions.UpgradeEntry{ return !colInfo.IsExits, nil }, } + +var upg_mo_pubs_add_account_name = versions.UpgradeEntry{ + Schema: catalog.MO_CATALOG, + TableName: catalog.MO_PUBS, + UpgType: versions.ADD_COLUMN, + UpgSql: "alter table mo_catalog.mo_pubs add column account_name varchar(300) after account_id", + CheckFunc: func(txn executor.TxnExecutor, accountId uint32) (bool, error) { + colInfo, err := versions.CheckTableColumn(txn, accountId, catalog.MO_CATALOG, catalog.MO_PUBS, "account_name") + if err != nil { + return false, err + } + return colInfo.IsExits, nil + }, + PostSql: "UPDATE mo_catalog.mo_pubs t1 INNER JOIN mo_catalog.mo_account t2 ON t1.account_id = t2.account_id SET t1.account_name = t2.account_name", +} + +var upg_mo_subs_add_sub_account_name = versions.UpgradeEntry{ + Schema: catalog.MO_CATALOG, + TableName: catalog.MO_SUBS, + UpgType: versions.ADD_COLUMN, + UpgSql: "alter table mo_catalog.mo_subs add column sub_account_name VARCHAR(300) NOT NULL after sub_account_id", + CheckFunc: func(txn executor.TxnExecutor, accountId uint32) (bool, error) { + colInfo, err := versions.CheckTableColumn(txn, accountId, catalog.MO_CATALOG, catalog.MO_SUBS, "sub_account_name") + if err != nil { + return false, err + } + return colInfo.IsExits, nil + }, + PostSql: "UPDATE mo_catalog.mo_subs t1 INNER JOIN mo_catalog.mo_account t2 ON t1.sub_account_id = t2.account_id SET t1.sub_account_name = t2.account_name", +} + +var upg_mo_subs_add_pub_account_id = versions.UpgradeEntry{ + Schema: catalog.MO_CATALOG, + TableName: catalog.MO_SUBS, + UpgType: versions.ADD_COLUMN, + UpgSql: "alter table mo_catalog.mo_subs add column pub_account_id INT NOT NULL after sub_time", + CheckFunc: func(txn executor.TxnExecutor, accountId uint32) (bool, error) { + colInfo, err := versions.CheckTableColumn(txn, accountId, catalog.MO_CATALOG, catalog.MO_SUBS, "pub_account_id") + if err != nil { + return false, err + } + return colInfo.IsExits, nil + }, + PostSql: "UPDATE mo_catalog.mo_subs t1 INNER JOIN mo_catalog.mo_account t2 ON t1.pub_account_name = t2.account_name SET t1.pub_account_id = t2.account_id", +} diff --git a/pkg/cdc/reader.go b/pkg/cdc/reader.go index b6a8bbcf4bdd7..d21d253558354 100644 --- a/pkg/cdc/reader.go +++ b/pkg/cdc/reader.go @@ -42,9 +42,10 @@ type tableReader struct { sinker Sinker wMarkUpdater IWatermarkUpdater tick *time.Ticker - resetWatermarkFunc func(*DbTableInfo) error initSnapshotSplitTxn bool runningReaders *sync.Map + startTs, endTs types.TS + noFull bool tableDef *plan.TableDef insTsColIdx, insCompositedPkColIdx int @@ -60,9 +61,10 @@ var NewTableReader = func( sinker Sinker, wMarkUpdater IWatermarkUpdater, tableDef *plan.TableDef, - resetWatermarkFunc func(*DbTableInfo) error, initSnapshotSplitTxn bool, runningReaders *sync.Map, + startTs, endTs types.TS, + noFull bool, ) Reader { reader := &tableReader{ cnTxnClient: cnTxnClient, @@ -73,9 +75,11 @@ var NewTableReader = func( sinker: sinker, wMarkUpdater: wMarkUpdater, tick: time.NewTicker(200 * time.Millisecond), - resetWatermarkFunc: resetWatermarkFunc, initSnapshotSplitTxn: initSnapshotSplitTxn, runningReaders: runningReaders, + startTs: startTs, + endTs: endTs, + noFull: noFull, tableDef: tableDef, } @@ -95,9 +99,7 @@ func (reader *tableReader) Close() { reader.sinker.Close() } -func (reader *tableReader) Run( - ctx context.Context, - ar *ActiveRoutine) { +func (reader *tableReader) Run(ctx context.Context, ar *ActiveRoutine) { var err error logutil.Infof("cdc tableReader(%v).Run: start", reader.info) defer func() { @@ -125,33 +127,24 @@ func (reader *tableReader) Run( if err = reader.readTable(ctx, ar); err != nil { logutil.Errorf("cdc tableReader(%v) failed, err: %v", reader.info, err) - - // if stale read, try to restart reader - if moerr.IsMoErrCode(err, moerr.ErrStaleRead) { - // reset sinker - reader.sinker.Reset() - // reset watermark - if err = reader.resetWatermarkFunc(reader.info); err != nil { - logutil.Errorf("cdc tableReader(%v) restart failed, err: %v", reader.info, err) - return - } - logutil.Errorf("cdc tableReader(%v) restart successfully", reader.info) - continue - } - - logutil.Errorf("cdc tableReader(%v) err is not stale read, quit", reader.info) return } } } -func (reader *tableReader) readTable( +var readTableWithTxn = func( + reader *tableReader, ctx context.Context, - ar *ActiveRoutine) (err error) { + txnOp client.TxnOperator, + packer *types.Packer, + ar *ActiveRoutine, +) (err error) { + return reader.readTableWithTxn(ctx, txnOp, packer, ar) +} - var txnOp client.TxnOperator +func (reader *tableReader) readTable(ctx context.Context, ar *ActiveRoutine) (err error) { //step1 : create an txnOp - txnOp, err = GetTxnOp(ctx, reader.cnEngine, reader.cnTxnClient, "readMultipleTables") + txnOp, err := GetTxnOp(ctx, reader.cnEngine, reader.cnTxnClient, "readMultipleTables") if err != nil { return err } @@ -164,8 +157,7 @@ func (reader *tableReader) readTable( ExitRunSql(txnOp) }() - err = GetTxn(ctx, reader.cnEngine, txnOp) - if err != nil { + if err = GetTxn(ctx, reader.cnEngine, txnOp); err != nil { return err } @@ -174,11 +166,25 @@ func (reader *tableReader) readTable( defer put.Put() //step2 : read table - err = reader.readTableWithTxn( - ctx, - txnOp, - packer, - ar) + err = readTableWithTxn(reader, ctx, txnOp, packer, ar) + // if stale read, try to reset watermark + if moerr.IsMoErrCode(err, moerr.ErrStaleRead) { + if !reader.noFull && !reader.startTs.IsEmpty() { + err = moerr.NewInternalErrorf(ctx, "cdc tableReader(%v) stale read, and startTs(%v) is set, end", reader.info, reader.startTs) + return + } + + // reset sinker + reader.sinker.Reset() + // reset watermark to startTs, will read from the beginning at next round + watermark := reader.startTs + if reader.noFull { + watermark = types.TimestampToTS(txnOp.SnapshotTS()) + } + reader.wMarkUpdater.UpdateMem(reader.info.SourceDbName, reader.info.SourceTblName, watermark) + logutil.Infof("cdc tableReader(%v) reset watermark success", reader.info) + err = nil + } return } @@ -199,7 +205,15 @@ func (reader *tableReader) readTableWithTxn( // from = last wmark // to = txn operator snapshot ts fromTs := reader.wMarkUpdater.GetFromMem(reader.info.SourceDbName, reader.info.SourceTblName) + if !reader.endTs.IsEmpty() && fromTs.GE(&reader.endTs) { + logutil.Debugf("current watermark(%v) >= endTs(%v), end", fromTs, reader.endTs) + return + } toTs := types.TimestampToTS(GetSnapshotTS(txnOp)) + if !reader.endTs.IsEmpty() && toTs.GT(&reader.endTs) { + toTs = reader.endTs + } + start := time.Now() changes, err = CollectChanges(ctx, rel, fromTs, toTs, reader.mp) v2.CdcReadDurationHistogram.Observe(time.Since(start).Seconds()) diff --git a/pkg/cdc/reader_test.go b/pkg/cdc/reader_test.go index 2df4c9864c60a..2cdb4e472f6bd 100644 --- a/pkg/cdc/reader_test.go +++ b/pkg/cdc/reader_test.go @@ -46,7 +46,6 @@ func TestNewTableReader(t *testing.T) { sinker Sinker wMarkUpdater *WatermarkUpdater tableDef *plan.TableDef - restartFunc func(*DbTableInfo) error runningReaders *sync.Map } @@ -86,11 +85,13 @@ func TestNewTableReader(t *testing.T) { tt.args.sinker, tt.args.wMarkUpdater, tt.args.tableDef, - tt.args.restartFunc, true, tt.args.runningReaders, + types.TS{}, + types.TS{}, + false, ), - "NewTableReader(%v,%v,%v,%v,%v,%v,%v,%v,%v)", + "NewTableReader(%v,%v,%v,%v,%v,%v,%v,%v)", tt.args.cnTxnClient, tt.args.cnEngine, tt.args.mp, @@ -98,8 +99,7 @@ func TestNewTableReader(t *testing.T) { tt.args.info, tt.args.sinker, tt.args.wMarkUpdater, - tt.args.tableDef, - tt.args.restartFunc) + tt.args.tableDef) }) } } @@ -114,7 +114,6 @@ func Test_tableReader_Run(t *testing.T) { sinker Sinker wMarkUpdater *WatermarkUpdater tick *time.Ticker - restartFunc func(*DbTableInfo) error insTsColIdx int insCompositedPkColIdx int delTsColIdx int @@ -236,7 +235,6 @@ func Test_tableReader_Run(t *testing.T) { sinker: tt.fields.sinker, wMarkUpdater: tt.fields.wMarkUpdater, tick: tt.fields.tick, - resetWatermarkFunc: tt.fields.restartFunc, insTsColIdx: tt.fields.insTsColIdx, insCompositedPkColIdx: tt.fields.insCompositedPkColIdx, delTsColIdx: tt.fields.delTsColIdx, @@ -248,80 +246,83 @@ func Test_tableReader_Run(t *testing.T) { } } -func Test_tableReader_Run_StaleRead(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second) - - stub := gostub.Stub(&GetTxnOp, +func Test_tableReader_readTable(t *testing.T) { + stub1 := gostub.Stub(&GetTxnOp, func(_ context.Context, _ engine.Engine, _ client.TxnClient, _ string) (client.TxnOperator, error) { - return nil, moerr.NewErrStaleReadNoCtx("", "") + return nil, nil }) - defer stub.Reset() + defer stub1.Reset() - // restart success - reader := &tableReader{ - tick: time.NewTicker(time.Millisecond * 300), - sinker: NewConsoleSinker(nil, nil), - resetWatermarkFunc: func(*DbTableInfo) error { return nil }, - runningReaders: &sync.Map{}, - info: &DbTableInfo{ - SourceDbName: "db1", - SourceTblName: "t1", - }, - } - reader.Run(ctx, NewCdcActiveRoutine()) - cancel() + stub2 := gostub.Stub(&FinishTxnOp, + func(ctx context.Context, inputErr error, txnOp client.TxnOperator, cnEngine engine.Engine) {}) + defer stub2.Reset() - // restart failed - ctx, cancel = context.WithTimeout(context.Background(), time.Second) - u := &WatermarkUpdater{ - accountId: 1, - taskId: uuid.New(), - ie: newWmMockSQLExecutor(), - watermarkMap: &sync.Map{}, - } - reader = &tableReader{ - tick: time.NewTicker(time.Millisecond * 300), - sinker: NewConsoleSinker(nil, nil), - info: &DbTableInfo{ - SourceDbName: "db1", - SourceTblName: "t1", - }, - wMarkUpdater: u, - resetWatermarkFunc: func(*DbTableInfo) error { return moerr.NewInternalErrorNoCtx("") }, - runningReaders: &sync.Map{}, - } - reader.Run(ctx, NewCdcActiveRoutine()) - cancel() -} + stub3 := gostub.Stub(&GetTxn, + func(ctx context.Context, cnEngine engine.Engine, txnOp client.TxnOperator) error { + return nil + }) + defer stub3.Reset() -func Test_tableReader_Run_NonStaleReadErr(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second) - defer cancel() + stub4 := gostub.Stub(&EnterRunSql, func(client.TxnOperator) {}) + defer stub4.Reset() - stub := gostub.Stub(&GetTxnOp, - func(_ context.Context, _ engine.Engine, _ client.TxnClient, _ string) (client.TxnOperator, error) { - return nil, moerr.NewInternalErrorNoCtx("this is a long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long long error message") - }) - defer stub.Reset() + stub5 := gostub.Stub(&ExitRunSql, func(client.TxnOperator) {}) + defer stub5.Reset() + + pool := fileservice.NewPool( + 128, + func() *types.Packer { + return types.NewPacker() + }, + func(packer *types.Packer) { + packer.Reset() + }, + func(packer *types.Packer) { + packer.Close() + }, + ) - u := &WatermarkUpdater{ - accountId: 1, - taskId: uuid.New(), - ie: newWmMockSQLExecutor(), - watermarkMap: &sync.Map{}, - } reader := &tableReader{ - tick: time.NewTicker(time.Millisecond * 300), - sinker: NewConsoleSinker(nil, nil), + packerPool: pool, + runningReaders: &sync.Map{}, + sinker: NewConsoleSinker(nil, nil), + wMarkUpdater: &WatermarkUpdater{ + accountId: 1, + taskId: uuid.New(), + ie: newWmMockSQLExecutor(), + watermarkMap: &sync.Map{}, + }, info: &DbTableInfo{ SourceDbName: "db1", SourceTblName: "t1", }, - wMarkUpdater: u, - resetWatermarkFunc: func(*DbTableInfo) error { return nil }, - runningReaders: &sync.Map{}, } - reader.Run(ctx, NewCdcActiveRoutine()) + ctx := context.Background() + ar := NewCdcActiveRoutine() + + // success + stub6 := gostub.Stub(&readTableWithTxn, func(*tableReader, context.Context, client.TxnOperator, *types.Packer, *ActiveRoutine) error { + return nil + }) + err := reader.readTable(ctx, ar) + assert.NoError(t, err) + stub6.Reset() + + // non-stale read error + stub7 := gostub.Stub(&readTableWithTxn, func(*tableReader, context.Context, client.TxnOperator, *types.Packer, *ActiveRoutine) error { + return moerr.NewInternalErrorNoCtx("") + }) + err = reader.readTable(ctx, ar) + assert.Error(t, err) + stub7.Reset() + + // stale read + stub8 := gostub.Stub(&readTableWithTxn, func(*tableReader, context.Context, client.TxnOperator, *types.Packer, *ActiveRoutine) error { + return moerr.NewErrStaleReadNoCtx("", "") + }) + err = reader.readTable(ctx, ar) + assert.NoError(t, err) + stub8.Reset() } func Test_tableReader_readTableWithTxn(t *testing.T) { @@ -362,6 +363,7 @@ func Test_tableReader_readTableWithTxn(t *testing.T) { insCompositedPkColIdx: 3, sinker: NewConsoleSinker(nil, nil), runningReaders: &sync.Map{}, + endTs: types.BuildTS(50, 0), } getRelationByIdStub := gostub.Stub(&GetRelationById, func(_ context.Context, _ engine.Engine, _ client.TxnOperator, _ uint64) (string, string, engine.Relation, error) { @@ -386,6 +388,10 @@ func Test_tableReader_readTableWithTxn(t *testing.T) { err := reader.readTableWithTxn(context.Background(), nil, packer, NewCdcActiveRoutine()) assert.NoError(t, err) + + reader.wMarkUpdater.UpdateMem("", "", types.BuildTS(50, 0)) + err = reader.readTableWithTxn(context.Background(), nil, packer, NewCdcActiveRoutine()) + assert.NoError(t, err) } var _ engine.ChangesHandle = new(testChangesHandle) diff --git a/pkg/cdc/types.go b/pkg/cdc/types.go index d632c16533114..d19ee3c46ed9f 100644 --- a/pkg/cdc/types.go +++ b/pkg/cdc/types.go @@ -60,6 +60,9 @@ const ( MaxSqlLength = "MaxSqlLength" DefaultMaxSqlLength = 4 * 1024 * 1024 + + StartTs = "StartTS" + EndTs = "EndTS" ) var ( diff --git a/pkg/common/pubsub/types.go b/pkg/common/pubsub/types.go index f8028c411361a..f25841b1687d0 100644 --- a/pkg/common/pubsub/types.go +++ b/pkg/common/pubsub/types.go @@ -92,8 +92,10 @@ func (pubInfo *PubInfo) GetCreateSql() string { type SubInfo struct { SubAccountId int32 + SubAccountName string SubName string SubTime string + PubAccountId int32 PubAccountName string PubName string PubDbName string diff --git a/pkg/container/types/packer.go b/pkg/container/types/packer.go index 17dbfe32d4e34..2feb023012abe 100644 --- a/pkg/container/types/packer.go +++ b/pkg/container/types/packer.go @@ -292,6 +292,12 @@ func (p *Packer) EncodeUuid(e Uuid) { p.putBytes(e[:]) } +func (p *Packer) EncodeObjectid(e *Objectid) { + p.putByte(objectIdCode) + p.putBytes(e[:SegmentidSize]) + p.encodeUint(uint64(e.Offset())) +} + func (p *Packer) GetBuf() []byte { return p.buffer } diff --git a/pkg/container/types/rowid.go b/pkg/container/types/rowid.go index cba7e39cae9cd..e875a2afb86e6 100644 --- a/pkg/container/types/rowid.go +++ b/pkg/container/types/rowid.go @@ -21,6 +21,7 @@ import ( "unsafe" "github.com/google/uuid" + "github.com/matrixorigin/matrixone/pkg/common/util" ) /* @@ -359,3 +360,9 @@ func (o *Objectid) LT(other *Objectid) bool { func (o *Objectid) GT(other *Objectid) bool { return o.Compare(other) > 0 } + +func (o *Objectid) Copy(offset uint16) Objectid { + ret := *o + copy(ret[SegmentidSize:], util.UnsafeToBytes(&offset)) + return ret +} diff --git a/pkg/container/types/tuple.go b/pkg/container/types/tuple.go index 551e30da87e7e..a0e00063928a4 100644 --- a/pkg/container/types/tuple.go +++ b/pkg/container/types/tuple.go @@ -37,6 +37,7 @@ import ( "unsafe" "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/common/util" "github.com/matrixorigin/matrixone/pkg/pb/plan" ) @@ -213,6 +214,7 @@ const ( enumCode = 0x50 // TODO: reorder the list to put timeCode next to date type code? bitCode = 0x51 uuidCode = 0x52 + objectIdCode = 0x53 ) var sizeLimits = []uint64{ @@ -426,6 +428,16 @@ func decodeUuid(b []byte) (Uuid, int) { return ret, 17 } +func decodeObjectid(b []byte) (Objectid, int) { + var ret Objectid + segid, pos1 := decodeUuid(b) + offset, pos2 := decodeUint(uint16Code, b[pos1:]) + u16 := offset.(uint16) + copy(ret[:], segid[:]) + copy(ret[UuidSize:], util.UnsafeToBytes(&u16)) + return ret, pos1 + pos2 +} + var DecodeTuple = decodeTuple func decodeTuple(b []byte) (Tuple, int, []T, error) { @@ -527,6 +539,9 @@ func decodeTuple(b []byte) (Tuple, int, []T, error) { schema = append(schema, T_uuid) el, off = decodeUuid(b[i:]) // off += 1 + case b[i] == objectIdCode: + schema = append(schema, T_Objectid) + el, off = decodeObjectid(b[i:]) default: return nil, i, nil, moerr.NewInternalErrorNoCtxf("unable to decode tuple element with unknown typecode %02x", b[i]) } diff --git a/pkg/container/types/types.go b/pkg/container/types/types.go index 139d9e188b884..fc119e22d8c25 100644 --- a/pkg/container/types/types.go +++ b/pkg/container/types/types.go @@ -81,9 +81,10 @@ const ( T_datalink T = 72 // Transaction TS - T_TS T = 100 - T_Rowid T = 101 - T_Blockid T = 102 + T_TS T = 100 + T_Rowid T = 101 + T_Blockid T = 102 + T_Objectid T = 103 // system family T_tuple T = 201 @@ -708,6 +709,8 @@ func (t T) String() string { return "DATALINK" case T_TS: return "TRANSACTION TIMESTAMP" + case T_Objectid: + return "OBJECTID" case T_Rowid: return "ROWID" case T_uuid: diff --git a/pkg/frontend/authenticate.go b/pkg/frontend/authenticate.go index c2f9db14ce95c..09a9315f2e3c4 100644 --- a/pkg/frontend/authenticate.go +++ b/pkg/frontend/authenticate.go @@ -3613,24 +3613,46 @@ func doDropAccount(ctx context.Context, bh BackgroundExec, ses *Session, da *dro return moerr.NewInternalErrorf(ctx, "can not delete the account %s", da.Name) } + checkAccount := func(accountName string) (accountId int64, version uint64, ok bool, err error) { + if sql, err = getSqlForCheckTenant(ctx, da.Name); err != nil { + return + } + + bh.ClearExecResultSet() + if err = bh.Exec(ctx, sql); err != nil { + return + } + + if erArray, err = getResultSet(ctx, bh); err != nil { + return + } + + if execResultArrayHasData(erArray) { + if accountId, err = erArray[0].GetInt64(ctx, 0, 0); err != nil { + return + } + if version, err = erArray[0].GetUint64(ctx, 0, 3); err != nil { + return + } + ok = true + } + + return + } + dropAccountFunc := func() (rtnErr error) { ses.Infof(ctx, "dropAccount %s sql: %s", da.Name, getAccountIdNamesSql) - _, nameInfoMap, rtnErr := getAccounts(ctx, bh, true) - if rtnErr != nil { - return rtnErr + if accountId, version, hasAccount, rtnErr = checkAccount(da.Name); rtnErr != nil { + return } - - //check the account exists or not - if _, ok := nameInfoMap[da.Name]; !ok { - //no such account - if !da.IfExists { //when the "IF EXISTS" is set, just skip it. + // check the account exists or not + if !hasAccount { + // when the "IF EXISTS" is set, just skip it. + if !da.IfExists { rtnErr = moerr.NewInternalErrorf(ctx, "there is no account %s", da.Name) } - hasAccount = false return } - accountId = int64(nameInfoMap[da.Name].Id) - version = nameInfoMap[da.Name].Version //drop tables of the tenant //NOTE!!!: single DDL drop statement per single transaction @@ -3661,7 +3683,7 @@ func doDropAccount(ctx context.Context, bh BackgroundExec, ses *Session, da *dro } for _, pubInfo := range pubInfos { ses.Infof(ctx, "dropAccount %s sql: %s", da.Name, pubInfo.PubName) - if rtnErr = dropPublication(deleteCtx, bh, true, pubInfo.PubName); rtnErr != nil { + if rtnErr = dropPublication(deleteCtx, bh, true, pubInfo.PubAccountName, pubInfo.PubName); rtnErr != nil { return } } @@ -3721,13 +3743,8 @@ func doDropAccount(ctx context.Context, bh BackgroundExec, ses *Session, da *dro return rtnErr } for _, subInfo := range subInfos { - pubAccInfo, ok := nameInfoMap[subInfo.PubAccountName] - if !ok { - continue - } - ses.Infof(ctx, "dropAccount %s sql: %s %s", da.Name, updatePubInfoAccountListFormat, subInfo.PubName) - if rtnErr = dropSubAccountNameInSubAccounts(deleteCtx, bh, pubAccInfo.Id, subInfo.PubName, da.Name); rtnErr != nil { + if rtnErr = dropSubAccountNameInSubAccounts(deleteCtx, bh, subInfo.PubAccountId, subInfo.PubName, da.Name); rtnErr != nil { return rtnErr } } @@ -7691,6 +7708,8 @@ func createSubscription(ctx context.Context, bh BackgroundExec, newTenant *Tenan for _, pubInfo := range pubInfos { subInfo := &pubsub.SubInfo{ SubAccountId: int32(newTenant.TenantID), + SubAccountName: newTenant.Tenant, + PubAccountId: accountId, PubAccountName: accIdInfoMap[accountId].Name, PubName: pubInfo.PubName, PubDbName: pubInfo.DbName, diff --git a/pkg/frontend/authenticate_test.go b/pkg/frontend/authenticate_test.go index b3aef63ec8023..fa3f1e102c747 100644 --- a/pkg/frontend/authenticate_test.go +++ b/pkg/frontend/authenticate_test.go @@ -7126,9 +7126,8 @@ func Test_doDropAccount(t *testing.T) { bh.sql2result["commit;"] = nil bh.sql2result["rollback;"] = nil - sql := getAccountIdNamesSql + " for update" + sql, _ := getSqlForCheckTenant(ctx, "acc") mrs := newMrsForGetAllAccounts([][]interface{}{ - {uint64(0), "sys", "open", uint64(1), nil}, {uint64(1), "acc", "open", uint64(1), nil}, }) bh.sql2result[sql] = mrs @@ -7187,7 +7186,7 @@ func Test_doDropAccount(t *testing.T) { bh.sql2result["commit;"] = nil bh.sql2result["rollback;"] = nil - sql := getAccountIdNamesSql + " for update" + sql, _ := getSqlForCheckTenant(ctx, "acc") bh.sql2result[sql] = newMrsForGetAllAccounts([][]interface{}{}) sql, _ = getSqlForDeleteAccountFromMoAccount(context.TODO(), mustUnboxExprStr(stmt.Name)) diff --git a/pkg/frontend/cdc.go b/pkg/frontend/cdc.go index f45d5803d6b0e..8fd0f22185379 100644 --- a/pkg/frontend/cdc.go +++ b/pkg/frontend/cdc.go @@ -85,6 +85,8 @@ const ( `sink_password, ` + `tables, ` + `filters, ` + + `start_ts, ` + + `end_ts, ` + `no_full, ` + `additional_config ` + `from ` + @@ -278,8 +280,8 @@ func handleCreateCdc(ses *Session, execCtx *ExecCtx, create *tree.CreateCDC) err func doCreateCdc(ctx context.Context, ses *Session, create *tree.CreateCDC) (err error) { service := ses.GetService() - ts := getPu(service).TaskService - if ts == nil { + taskService := getPu(service).TaskService + if taskService == nil { return moerr.NewInternalError(ctx, "no task service is found") } @@ -315,6 +317,23 @@ func doCreateCdc(ctx context.Context, ses *Session, create *tree.CreateCDC) (err return moerr.NewInternalErrorf(ctx, "invalid exclude expression: %s, err: %v", exclude, err) } + var ts time.Time + startTs := cdcTaskOptionsMap[cdc2.StartTs] + if startTs != "" { + if ts, err = parseTimestamp(startTs, ses.timeZone); err != nil { + return moerr.NewInternalErrorf(ctx, "invalid startTs: %s, supported timestamp format: `%s`, or `%s`", startTs, time.DateTime, time.RFC3339) + } + startTs = ts.Format(time.RFC3339) + } + + endTs := cdcTaskOptionsMap[cdc2.EndTs] + if endTs != "" { + if ts, err = parseTimestamp(endTs, ses.timeZone); err != nil { + return moerr.NewInternalErrorf(ctx, "invalid endTs: %s, supported timestamp format: `%s`, or `%s`", endTs, time.DateTime, time.RFC3339) + } + endTs = ts.Format(time.RFC3339) + } + //step 4: check source uri format and strip password jsonSrcUri, _, err := extractUriInfo(ctx, create.SourceUri, cdc2.SourceUriPrefix) if err != nil { @@ -434,8 +453,8 @@ func doCreateCdc(ctx context.Context, ses *Session, create *tree.CreateCDC) (err "", cdc2.SASCommon, cdc2.SASCommon, - "", //1.3 does not support startTs - "", //1.3 does not support endTs + startTs, + endTs, cdcTaskOptionsMap["ConfigFile"], time.Now().UTC(), CdcRunning, @@ -458,7 +477,7 @@ func doCreateCdc(ctx context.Context, ses *Session, create *tree.CreateCDC) (err return int(cdcTaskRowsAffected), nil } - if _, err = ts.AddCdcTask(ctx, cdcTaskMetadata(cdcId.String()), details, addCdcTaskCallback); err != nil { + if _, err = taskService.AddCdcTask(ctx, cdcTaskMetadata(cdcId.String()), details, addCdcTaskCallback); err != nil { return err } return @@ -727,7 +746,7 @@ type CdcTask struct { sinkUri cdc2.UriInfo tables cdc2.PatternTuples exclude *regexp.Regexp - startTs types.TS + startTs, endTs types.TS noFull bool additionalConfig map[string]interface{} @@ -917,23 +936,6 @@ func (cdc *CdcTask) Cancel() (err error) { return } -func (cdc *CdcTask) resetWatermarkForTable(info *cdc2.DbTableInfo) (err error) { - dbName, tblName := info.SourceDbName, info.SourceTblName - // delete old watermark of table - cdc.watermarkUpdater.DeleteFromMem(dbName, tblName) - if err = cdc.watermarkUpdater.DeleteFromDb(dbName, tblName); err != nil { - return - } - - // use start_ts as init watermark - // TODO handle no_full - if err = cdc.watermarkUpdater.InsertIntoDb(info, cdc.startTs); err != nil { - return - } - cdc.watermarkUpdater.UpdateMem(dbName, tblName, cdc.startTs) - return -} - func (cdc *CdcTask) initAesKeyByInternalExecutor(ctx context.Context, accountId uint32) (err error) { if len(cdc2.AesKey) > 0 { return nil @@ -1098,9 +1100,11 @@ func (cdc *CdcTask) addExecPipelineForTable(ctx context.Context, info *cdc2.DbTa sinker, cdc.watermarkUpdater, tableDef, - cdc.resetWatermarkForTable, cdc.additionalConfig[cdc2.InitSnapshotSplitTxn].(bool), cdc.runningReaders, + cdc.startTs, + cdc.endTs, + cdc.noFull, ) go reader.Run(ctx, cdc.activeRoutine) @@ -1181,18 +1185,50 @@ func (cdc *CdcTask) retrieveCdcTask(ctx context.Context) error { } } + convertToTs := func(tsStr string) (types.TS, error) { + t, err := parseTimestamp(tsStr, nil) + if err != nil { + return types.TS{}, err + } + + return types.BuildTS(t.UnixNano(), 0), nil + } + + // startTs + startTs, err := res.GetString(ctx, 0, 5) + if err != nil { + return err + } + if startTs == "" { + cdc.startTs = types.TS{} + } else { + if cdc.startTs, err = convertToTs(startTs); err != nil { + return err + } + } + + // endTs + endTs, err := res.GetString(ctx, 0, 6) + if err != nil { + return err + } + if endTs == "" { + cdc.endTs = types.TS{} + } else { + if cdc.endTs, err = convertToTs(endTs); err != nil { + return err + } + } + // noFull - noFull, err := res.GetString(ctx, 0, 5) + noFull, err := res.GetString(ctx, 0, 7) if err != nil { return err } cdc.noFull, _ = strconv.ParseBool(noFull) - // startTs - cdc.startTs = types.TS{} - // additionalConfig - additionalConfigStr, err := res.GetString(ctx, 0, 6) + additionalConfigStr, err := res.GetString(ctx, 0, 8) if err != nil { return err } @@ -1649,3 +1685,18 @@ var initAesKeyBySqlExecutor = func(ctx context.Context, executor taskservice.Sql cdc2.AesKey, err = cdc2.AesCFBDecodeWithKey(ctx, encryptedKey, []byte(getGlobalPuWrapper(service).SV.KeyEncryptionKey)) return } + +func parseTimestamp(tsStr string, tz *time.Location) (ts time.Time, err error) { + if tsStr == "" { + return + } + + if tz != nil { + if ts, err = time.ParseInLocation(time.DateTime, tsStr, tz); err == nil { + return + } + } + + ts, err = time.Parse(time.RFC3339, tsStr) + return +} diff --git a/pkg/frontend/cdc_test.go b/pkg/frontend/cdc_test.go index 087169feb1ce3..c1f54c858a065 100644 --- a/pkg/frontend/cdc_test.go +++ b/pkg/frontend/cdc_test.go @@ -83,7 +83,7 @@ func Test_newCdcSqlFormat(t *testing.T) { assert.Equal(t, wantSql, sql) sql2 := getSqlForRetrievingCdcTask(3, id) - wantSql2 := "select sink_uri, sink_type, sink_password, tables, filters, no_full, additional_config from mo_catalog.mo_cdc_task where account_id = 3 and task_id = \"019111fd-aed1-70c0-8760-9abadd8f0f4a\"" + wantSql2 := "select sink_uri, sink_type, sink_password, tables, filters, start_ts, end_ts, no_full, additional_config from mo_catalog.mo_cdc_task where account_id = 3 and task_id = \"019111fd-aed1-70c0-8760-9abadd8f0f4a\"" assert.Equal(t, wantSql2, sql2) sql3 := getSqlForDbIdAndTableId(10, "db", "t1") @@ -404,6 +404,10 @@ func Test_handleCreateCdc(t *testing.T) { fmt.Sprintf("%d", cdc2.DefaultMaxSqlLength), cdc2.SendSqlTimeout, cdc2.DefaultSendSqlTimeout, + cdc2.StartTs, + "2025-01-03 15:20:00", + cdc2.EndTs, + "2025-01-03 16:20:00", }, } @@ -449,6 +453,51 @@ func Test_handleCreateCdc(t *testing.T) { } } +func Test_doCreateCdc_invalidStartTs(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + ses := newTestSession(t, ctrl) + defer ses.Close() + + pu := config.ParameterUnit{} + pu.TaskService = &testTaskService{} + setPu("", &pu) + + stubCheckPitr := gostub.Stub(&checkPitr, func(ctx context.Context, bh BackgroundExec, accName string, pts *cdc2.PatternTuples) error { + return nil + }) + defer stubCheckPitr.Reset() + + create := &tree.CreateCDC{ + IfNotExists: false, + TaskName: "task1", + SourceUri: "mysql://root:111@127.0.0.1:6001", + SinkType: cdc2.MysqlSink, + SinkUri: "mysql://root:111@127.0.0.1:3306", + Tables: "db1.t1:db1.t1,db1.t2", + Option: []string{ + "Level", + cdc2.TableLevel, + "Account", + sysAccountName, + "Exclude", + "db2.t3,db2.t4", + cdc2.InitSnapshotSplitTxn, + "false", + cdc2.MaxSqlLength, + fmt.Sprintf("%d", cdc2.DefaultMaxSqlLength), + cdc2.SendSqlTimeout, + cdc2.DefaultSendSqlTimeout, + cdc2.StartTs, + "123456", + }, + } + + err := doCreateCdc(context.Background(), ses, create) + assert.Error(t, err) +} + type testTaskData struct { metadata task.TaskMetadata details *task.Details @@ -618,6 +667,8 @@ func (tie *testIE) Query(ctx context.Context, s string, options ie.SessionOverri sinkPwd := "" tables := "" filters := "" + startTs := "" + endTs := "" noFull := "" splitTxn := "" err = rows.Scan( @@ -626,6 +677,8 @@ func (tie *testIE) Query(ctx context.Context, s string, options ie.SessionOverri &sinkPwd, &tables, &filters, + &startTs, + &endTs, &noFull, &splitTxn) if err != nil { @@ -636,6 +689,8 @@ func (tie *testIE) Query(ctx context.Context, s string, options ie.SessionOverri rowValues = append(rowValues, sinkPwd) rowValues = append(rowValues, tables) rowValues = append(rowValues, filters) + rowValues = append(rowValues, startTs) + rowValues = append(rowValues, endTs) rowValues = append(rowValues, noFull) rowValues = append(rowValues, splitTxn) } else if idx == mSqlIdx2 { @@ -849,7 +904,7 @@ func TestRegisterCdcExecutor(t *testing.T) { db, mock, err := sqlmock.New() assert.NoError(t, err) /////////mock sql result - sql1 := `select sink_uri, sink_type, sink_password, tables, filters, no_full, additional_config from mo_catalog.mo_cdc_task where account_id = 0 and task_id = "00000000-0000-0000-0000-000000000000"` + sql1 := `select sink_uri, sink_type, sink_password, tables, filters, start_ts, end_ts, no_full, additional_config from mo_catalog.mo_cdc_task where account_id = 0 and task_id = "00000000-0000-0000-0000-000000000000"` mock.ExpectQuery(sql1).WillReturnRows(sqlmock.NewRows( []string{ "sink_uri", @@ -857,6 +912,8 @@ func TestRegisterCdcExecutor(t *testing.T) { "sink_password", "tables", "filters", + "start_ts", + "end_ts", "no_full", "additional_config", }, @@ -866,6 +923,8 @@ func TestRegisterCdcExecutor(t *testing.T) { pwd, tables, filters, + "", + "", true, fmt.Sprintf("{\"%s\":%v,\"%s\":\"%s\",\"%s\":%v}", cdc2.InitSnapshotSplitTxn, cdc2.DefaultInitSnapshotSplitTxn, @@ -2139,24 +2198,6 @@ func Test_handleShowCdc(t *testing.T) { } } -func TestCdcTask_ResetWatermarkForTable(t *testing.T) { - cdc := &CdcTask{ - watermarkUpdater: &mockWatermarkUpdater{}, - } - - info := &cdc2.DbTableInfo{ - SourceDbId: 0, - SourceDbName: "", - SourceTblId: 0, - SourceTblName: "", - SourceCreateSql: "", - SinkDbName: "", - SinkTblName: "", - } - - assert.NoError(t, cdc.resetWatermarkForTable(info)) -} - func TestCdcTask_Resume(t *testing.T) { cdc := &CdcTask{ activeRoutine: cdc2.NewCdcActiveRoutine(), @@ -2273,7 +2314,7 @@ func TestCdcTask_retrieveCdcTask(t *testing.T) { db, mock, err := sqlmock.New() assert.NoError(t, err) - sqlx := "select sink_uri, sink_type, sink_password, tables, filters, no_full, additional_config from mo_catalog.mo_cdc_task where account_id = .* and task_id =.*" + sqlx := "select sink_uri, sink_type, sink_password, tables, filters, start_ts, end_ts, no_full, additional_config from mo_catalog.mo_cdc_task where account_id = .* and task_id =.*" sinkUri, err := cdc2.JsonEncode(&cdc2.UriInfo{ User: "root", Ip: "127.0.0.1", @@ -2304,6 +2345,8 @@ func TestCdcTask_retrieveCdcTask(t *testing.T) { "sink_password", "tables", "filters", + "start_ts", + "end_ts", "no_full", "additional_config", }, @@ -2313,6 +2356,8 @@ func TestCdcTask_retrieveCdcTask(t *testing.T) { pwd, tables, filters, + "2006-01-02T15:04:05-07:00", + "2006-01-02T15:04:05-07:00", true, "{\"InitSnapshotSplitTxn\": false}", ), @@ -2707,7 +2752,9 @@ func (m mockWatermarkUpdater) GetFromDb(dbName, tblName string) (watermark types return } -func (m mockWatermarkUpdater) UpdateMem(string, string, types.TS) {} +func (m mockWatermarkUpdater) UpdateMem(string, string, types.TS) { + +} func (m mockWatermarkUpdater) DeleteFromMem(string, string) {} @@ -2715,11 +2762,6 @@ func (m mockWatermarkUpdater) DeleteFromDb(string, string) error { return nil } -func (m mockWatermarkUpdater) DeleteAllFromDb() error { - //TODO implement me - panic("implement me") -} - func (m mockWatermarkUpdater) SaveErrMsg(string, string, string) error { return nil } @@ -2812,8 +2854,9 @@ func TestCdcTask_addExecPipelineForTable(t *testing.T) { }) defer stubSinker.Reset() - stubReader := gostub.Stub(&cdc2.NewTableReader, func(client.TxnClient, engine.Engine, *mpool.MPool, *fileservice.Pool[*types.Packer], - *cdc2.DbTableInfo, cdc2.Sinker, cdc2.IWatermarkUpdater, *plan.TableDef, func(*cdc2.DbTableInfo) error, bool, *sync.Map) cdc2.Reader { + stubReader := gostub.Stub(&cdc2.NewTableReader, func(client.TxnClient, engine.Engine, *mpool.MPool, + *fileservice.Pool[*types.Packer], *cdc2.DbTableInfo, cdc2.Sinker, cdc2.IWatermarkUpdater, *plan.TableDef, bool, + *sync.Map, types.TS, types.TS, bool) cdc2.Reader { return &mockReader{} }) defer stubReader.Reset() @@ -2855,3 +2898,11 @@ func TestCdcTask_checkPitr(t *testing.T) { err := checkPitr(context.Background(), nil, "acc1", pts) assert.Error(t, err) } + +func Test_parseTimestamp(t *testing.T) { + _, err := parseTimestamp("2006-01-02 15:04:05", time.Local) + assert.NoError(t, err) + + _, err = parseTimestamp("2006-01-02T15:04:05-07:00", nil) + assert.NoError(t, err) +} diff --git a/pkg/frontend/mysql_protocol.go b/pkg/frontend/mysql_protocol.go index e1ef0b69ad3df..629f95c0ed2cb 100644 --- a/pkg/frontend/mysql_protocol.go +++ b/pkg/frontend/mysql_protocol.go @@ -249,14 +249,14 @@ type MysqlProtocolImpl struct { maxClientPacketSize uint32 //the user of the client - username string + username atomic.Value // opaque authentication response data generated by Authentication Method // indicated by the plugin name field. authResponse []byte //the default database for the client - database string + database atomic.Value // Connection attributes are key-value pairs that application programs // can pass to the server at connect time. @@ -523,15 +523,15 @@ func (mp *MysqlProtocolImpl) SetSequenceID(value uint8) { } func (mp *MysqlProtocolImpl) GetDatabaseName() string { - mp.m.Lock() - defer mp.m.Unlock() - return mp.database + val := mp.database.Load() + if val == nil { + return "" + } + return val.(string) } func (mp *MysqlProtocolImpl) SetDatabaseName(s string) { - mp.m.Lock() - defer mp.m.Unlock() - mp.database = s + mp.database.Store(s) } func (mp *MysqlProtocolImpl) GetAuthString() []byte { @@ -547,15 +547,15 @@ func (mp *MysqlProtocolImpl) GetAuthResponse() []byte { } func (mp *MysqlProtocolImpl) GetUserName() string { - mp.m.Lock() - defer mp.m.Unlock() - return mp.username + val := mp.username.Load() + if val == nil { + return "" + } + return val.(string) } func (mp *MysqlProtocolImpl) SetUserName(s string) { - mp.m.Lock() - defer mp.m.Unlock() - mp.username = s + mp.username.Store(s) } const defaultTcp4PackageSize = 1<<14 - 66 @@ -1521,8 +1521,8 @@ func (mp *MysqlProtocolImpl) HandleHandshake(ctx context.Context, payload []byte } mp.maxClientPacketSize = resp41.maxPacketSize - mp.username = resp41.username - mp.database = resp41.database + mp.username.Store(resp41.username) + mp.database.Store(resp41.database) mp.connectAttrs = resp41.connectAttrs } else { var resp320 response320 @@ -1543,8 +1543,8 @@ func (mp *MysqlProtocolImpl) HandleHandshake(ctx context.Context, payload []byte mp.charset = "utf8mb4" mp.maxClientPacketSize = resp320.maxPacketSize - mp.username = resp320.username - mp.database = resp320.database + mp.username.Store(resp320.username) + mp.database.Store(resp320.database) } return false, nil } @@ -1560,7 +1560,7 @@ func (mp *MysqlProtocolImpl) Authenticate(ctx context.Context) error { ses.Debugf(ctx, "authenticate user") if err := mp.authenticateUser(ctx, mp.authResponse); err != nil { ses.Errorf(ctx, "authenticate user failed.error:%v", err) - errorCode, sqlState, msg := RewriteError(err, mp.username) + errorCode, sqlState, msg := RewriteError(err, mp.GetUserName()) ses.timestampMap[TSSendErrPacketStart] = time.Now() err2 := mp.sendErrPacket(errorCode, sqlState, msg) ses.timestampMap[TSSendErrPacketEnd] = time.Now() diff --git a/pkg/frontend/predefined.go b/pkg/frontend/predefined.go index b82cc16f32dea..977a31c731df9 100644 --- a/pkg/frontend/predefined.go +++ b/pkg/frontend/predefined.go @@ -163,6 +163,7 @@ var ( MoCatalogMoPubsDDL = `create table mo_catalog.mo_pubs ( account_id int not null, + account_name varchar(300), pub_name varchar(64), database_name varchar(5000), database_id bigint unsigned, @@ -179,8 +180,10 @@ var ( MoCatalogMoSubsDDL = `create table mo_catalog.mo_subs ( sub_account_id INT NOT NULL, + sub_account_name VARCHAR(300) NOT NULL, sub_name VARCHAR(5000) DEFAULT NULL, sub_time TIMESTAMP DEFAULT NULL, + pub_account_id INT NOT NULL, pub_account_name VARCHAR(300) NOT NULL, pub_name VARCHAR(64) NOT NULL, pub_database VARCHAR(5000) NOT NULL, diff --git a/pkg/frontend/publication_subscription.go b/pkg/frontend/publication_subscription.go index 35b0468578062..b09dfce7b031f 100644 --- a/pkg/frontend/publication_subscription.go +++ b/pkg/frontend/publication_subscription.go @@ -36,19 +36,19 @@ const ( // account getAccountIdNamesSql = "select account_id, account_name, status, version, suspended_time from mo_catalog.mo_account where 1=1" // pub - insertIntoMoPubsFormat = `insert into mo_catalog.mo_pubs (account_id, pub_name, database_name, database_id, all_table, table_list, account_list, created_time, owner, creator, comment) values (%d, '%s', '%s', %d, %t, '%s', '%s', now(), %d, %d, '%s');` - getAllPubInfoSql = "select account_id, pub_name, database_name, database_id, table_list, account_list, created_time, update_time, owner, creator, comment from mo_catalog.mo_pubs" - getPubInfoSql = "select account_id, pub_name, database_name, database_id, table_list, account_list, created_time, update_time, owner, creator, comment from mo_catalog.mo_pubs where account_id = %d" + insertIntoMoPubsFormat = `insert into mo_catalog.mo_pubs (account_id, account_name, pub_name, database_name, database_id, all_table, table_list, account_list, created_time, owner, creator, comment) values (%d, '%s', '%s', '%s', %d, %t, '%s', '%s', now(), %d, %d, '%s');` + getAllPubInfoSql = "select account_id, account_name, pub_name, database_name, database_id, table_list, account_list, created_time, update_time, owner, creator, comment from mo_catalog.mo_pubs" + getPubInfoSql = "select account_id, account_name, pub_name, database_name, database_id, table_list, account_list, created_time, update_time, owner, creator, comment from mo_catalog.mo_pubs where account_id = %d" updatePubInfoFormat = `update mo_catalog.mo_pubs set account_list = '%s', comment = '%s', database_name = '%s', database_id = %d, update_time = now(), table_list = '%s' where account_id = %d and pub_name = '%s';` updatePubInfoAccountListFormat = `update mo_catalog.mo_pubs set account_list = '%s' where account_id = %d and pub_name = '%s';` dropPubFormat = `delete from mo_catalog.mo_pubs where account_id = %d and pub_name = '%s';` getDbPubCountFormat = `select count(1) from mo_catalog.mo_pubs where account_id = %d and database_name = '%s';` // sub - insertIntoMoSubsFormat = "insert into mo_catalog.mo_subs (sub_account_id, pub_account_name, pub_name, pub_database, pub_tables, pub_time, pub_comment, status) values (%d, '%s', '%s', '%s', '%s', now(), '%s', %d)" - batchInsertIntoMoSubsFormat = "insert into mo_catalog.mo_subs (sub_account_id, pub_account_name, pub_name, pub_database, pub_tables, pub_time, pub_comment, status) values %s" + insertIntoMoSubsFormat = "insert into mo_catalog.mo_subs (sub_account_id, sub_account_name, pub_account_id, pub_account_name, pub_name, pub_database, pub_tables, pub_time, pub_comment, status) values (%d, '%s', %d, '%s', '%s', '%s', '%s', now(), '%s', %d)" + batchInsertIntoMoSubsFormat = "insert into mo_catalog.mo_subs (sub_account_id, sub_account_name, pub_account_id, pub_account_name, pub_name, pub_database, pub_tables, pub_time, pub_comment, status) values %s" batchUpdateMoSubsFormat = "update mo_catalog.mo_subs set pub_database='%s', pub_tables='%s', pub_time=now(), pub_comment='%s', status=%d where pub_account_name = '%s' and pub_name = '%s' and sub_account_id in (%s)" batchDeleteMoSubsFormat = "delete from mo_catalog.mo_subs where pub_account_name = '%s' and pub_name = '%s' and sub_account_id in (%s)" - getSubsSql = "select sub_account_id, sub_name, sub_time, pub_account_name, pub_name, pub_database, pub_tables, pub_time, pub_comment, status from mo_catalog.mo_subs where 1=1" + getSubsSql = "select sub_account_id, sub_account_name, sub_name, sub_time, pub_account_id, pub_account_name, pub_name, pub_database, pub_tables, pub_time, pub_comment, status from mo_catalog.mo_subs where 1=1" deleteMoSubsRecordsBySubAccountIdFormat = "delete from mo_catalog.mo_subs where sub_account_id = %d" // database getDbIdAndTypFormat = `select dat_id,dat_type from mo_catalog.mo_database where datname = '%s' and account_id = %d;` @@ -246,7 +246,7 @@ func createPublication(ctx context.Context, bh BackgroundExec, cp *tree.CreatePu } } - sql, err = getSqlForInsertIntoMoPubs(ctx, pubName, dbName, dbId, len(cp.Table) == 0, tablesStr, accountNamesStr, comment, true) + sql, err = getSqlForInsertIntoMoPubs(ctx, accountId, accountName, pubName, dbName, dbId, len(cp.Table) == 0, tablesStr, accountNamesStr, comment, true) if err != nil { return } @@ -306,9 +306,9 @@ func createPublication(ctx context.Context, bh BackgroundExec, cp *tree.CreatePu } if err = batchInsertMoSubs( ctx, bh, - dbName, tablesStr, comment, - accountName, pubName, - insertSubAccounts, + int32(accountId), accountName, + pubName, dbName, tablesStr, comment, + insertSubAccounts, accIdInfoMap, ); err != nil { return } @@ -516,9 +516,9 @@ func doAlterPublication(ctx context.Context, ses *Session, ap *tree.AlterPublica } if err = batchInsertMoSubs( ctx, bh, - dbName, tablesStr, comment, - accountName, pubName, - insertSubAccounts, + int32(accountId), accountName, + pubName, dbName, tablesStr, comment, + insertSubAccounts, accIdInfoMap, ); err != nil { return } @@ -547,23 +547,13 @@ func doDropPublication(ctx context.Context, ses *Session, dp *tree.DropPublicati err = finishTxn(ctx, bh, err) }() - return dropPublication(ctx, bh, dp.IfExists, string(dp.Name)) + return dropPublication(ctx, bh, dp.IfExists, tenantInfo.Tenant, string(dp.Name)) } // dropPublication drops a publication, bh should be in a transaction -func dropPublication(ctx context.Context, bh BackgroundExec, ifExists bool, pubName string) (err error) { +func dropPublication(ctx context.Context, bh BackgroundExec, ifExists bool, accountName string, pubName string) (err error) { var sql string - accIdInfoMap, _, err := getAccounts(ctx, bh, false) - if err != nil { - return - } - accountId, err := defines.GetAccountId(ctx) - if err != nil { - return err - } - accountName := accIdInfoMap[int32(accountId)].Name - pub, err := getPubInfo(ctx, bh, pubName) if err != nil { return err @@ -583,7 +573,7 @@ func dropPublication(ctx context.Context, bh BackgroundExec, ifExists bool, pubN } // subAccountName -> SubInfo map - subInfos, err := getSubInfosFromPub(ctx, bh, accIdInfoMap[int32(accountId)].Name, pubName, false) + subInfos, err := getSubInfosFromPub(ctx, bh, accountName, pubName, false) if err != nil { return err } @@ -697,6 +687,7 @@ func getAccounts(ctx context.Context, bh BackgroundExec, forUpdate bool) (idInfo func extractPubInfosFromExecResult(ctx context.Context, erArray []ExecResult) (pubInfos []*pubsub.PubInfo, err error) { var ( accountId int64 + accountName string pubName string dbName string dbId uint64 @@ -714,45 +705,49 @@ func extractPubInfosFromExecResult(ctx context.Context, erArray []ExecResult) (p if accountId, err = result.GetInt64(ctx, i, 0); err != nil { return } - if pubName, err = result.GetString(ctx, i, 1); err != nil { + if accountName, err = result.GetString(ctx, i, 1); err != nil { return } - if dbName, err = result.GetString(ctx, i, 2); err != nil { + if pubName, err = result.GetString(ctx, i, 2); err != nil { return } - if dbId, err = result.GetUint64(ctx, i, 3); err != nil { + if dbName, err = result.GetString(ctx, i, 3); err != nil { return } - if tablesStr, err = result.GetString(ctx, i, 4); err != nil { + if dbId, err = result.GetUint64(ctx, i, 4); err != nil { return } - if accountNamesStr, err = result.GetString(ctx, i, 5); err != nil { + if tablesStr, err = result.GetString(ctx, i, 5); err != nil { return } - if createTime, err = result.GetString(ctx, i, 6); err != nil { + if accountNamesStr, err = result.GetString(ctx, i, 6); err != nil { return } - if isNull, err = result.ColumnIsNull(ctx, i, 7); err != nil { + if createTime, err = result.GetString(ctx, i, 7); err != nil { + return + } + if isNull, err = result.ColumnIsNull(ctx, i, 8); err != nil { return } else if !isNull { - if updateTime, err = result.GetString(ctx, i, 7); err != nil { + if updateTime, err = result.GetString(ctx, i, 8); err != nil { return } } else { updateTime = "" } - if owner, err = result.GetUint64(ctx, i, 8); err != nil { + if owner, err = result.GetUint64(ctx, i, 9); err != nil { return } - if creator, err = result.GetUint64(ctx, i, 9); err != nil { + if creator, err = result.GetUint64(ctx, i, 10); err != nil { return } - if comment, err = result.GetString(ctx, i, 10); err != nil { + if comment, err = result.GetString(ctx, i, 11); err != nil { return } pubInfos = append(pubInfos, &pubsub.PubInfo{ PubAccountId: uint32(accountId), + PubAccountName: accountName, PubName: pubName, DbName: dbName, DbId: dbId, @@ -880,8 +875,10 @@ func getAllPubInfosBySnapshotName(ctx context.Context, bh BackgroundExec, snapsh func extractSubInfosFromExecResult(ctx context.Context, erArray []ExecResult) (subInfos []*pubsub.SubInfo, err error) { var ( subAccountId int64 + subAccountName string subName string subTime string + pubAccountId int64 pubAccountName string pubName string pubDbName string @@ -896,50 +893,58 @@ func extractSubInfosFromExecResult(ctx context.Context, erArray []ExecResult) (s if subAccountId, err = result.GetInt64(ctx, i, 0); err != nil { return } - if isNull, err = result.ColumnIsNull(ctx, i, 1); err != nil { + if subAccountName, err = result.GetString(ctx, i, 1); err != nil { + return + } + if isNull, err = result.ColumnIsNull(ctx, i, 2); err != nil { return } else if !isNull { - if subName, err = result.GetString(ctx, i, 1); err != nil { + if subName, err = result.GetString(ctx, i, 2); err != nil { return } } else { subName = "" } - if isNull, err = result.ColumnIsNull(ctx, i, 2); err != nil { + if isNull, err = result.ColumnIsNull(ctx, i, 3); err != nil { return } else if !isNull { - if subTime, err = result.GetString(ctx, i, 2); err != nil { + if subTime, err = result.GetString(ctx, i, 3); err != nil { return } } else { subTime = "" } - if pubAccountName, err = result.GetString(ctx, i, 3); err != nil { + if pubAccountId, err = result.GetInt64(ctx, i, 4); err != nil { + return + } + if pubAccountName, err = result.GetString(ctx, i, 5); err != nil { return } - if pubName, err = result.GetString(ctx, i, 4); err != nil { + if pubName, err = result.GetString(ctx, i, 6); err != nil { return } - if pubDbName, err = result.GetString(ctx, i, 5); err != nil { + if pubDbName, err = result.GetString(ctx, i, 7); err != nil { return } - if pubTables, err = result.GetString(ctx, i, 6); err != nil { + if pubTables, err = result.GetString(ctx, i, 8); err != nil { return } - if pubTime, err = result.GetString(ctx, i, 7); err != nil { + if pubTime, err = result.GetString(ctx, i, 9); err != nil { return } - if pubComment, err = result.GetString(ctx, i, 8); err != nil { + if pubComment, err = result.GetString(ctx, i, 10); err != nil { return } - if status, err = result.GetInt64(ctx, i, 9); err != nil { + if status, err = result.GetInt64(ctx, i, 11); err != nil { return } subInfos = append(subInfos, &pubsub.SubInfo{ SubAccountId: int32(subAccountId), + SubAccountName: subAccountName, SubName: subName, SubTime: subTime, + PubAccountId: int32(pubAccountId), PubAccountName: pubAccountName, PubName: pubName, PubDbName: pubDbName, @@ -1352,16 +1357,17 @@ func getSqlForUpdatePubInfo(ctx context.Context, pubName string, accountList str func insertMoSubs(ctx context.Context, bh BackgroundExec, subInfo *pubsub.SubInfo) (err error) { ctx = defines.AttachAccountId(ctx, catalog.System_Account) - sql := fmt.Sprintf(insertIntoMoSubsFormat, subInfo.SubAccountId, subInfo.PubAccountName, subInfo.PubName, subInfo.PubDbName, subInfo.PubTables, subInfo.PubComment, subInfo.Status) + sql := fmt.Sprintf(insertIntoMoSubsFormat, subInfo.SubAccountId, subInfo.SubAccountName, subInfo.PubAccountId, subInfo.PubAccountName, subInfo.PubName, subInfo.PubDbName, subInfo.PubTables, subInfo.PubComment, subInfo.Status) return bh.Exec(ctx, sql) } func batchInsertMoSubs( ctx context.Context, bh BackgroundExec, - pubDbName, pubTables, pubComment string, - pubAccountName, pubName string, + pubAccountId int32, pubAccountName, + pubName, pubDbName, pubTables, pubComment string, accIds []int32, + accIdInfoMap map[int32]*pubsub.AccountInfo, ) (err error) { if len(accIds) == 0 { return @@ -1369,13 +1375,13 @@ func batchInsertMoSubs( ctx = defines.AttachAccountId(ctx, catalog.System_Account) values := make([]string, 0, len(accIds)) - // sub_account_id, pub_account_name, pub_name, pub_database, pub_tables, pub_time, pub_comment, status - valuesFormat := "(%d, '%s', '%s', '%s', '%s', now(), '%s', %d)" + // sub_account_id, sub_account_name, pub_account_id, pub_account_name, pub_name, pub_database, pub_tables, pub_time, pub_comment, status + valuesFormat := "(%d, '%s', %d, '%s', '%s', '%s', '%s', now(), '%s', %d)" for _, accId := range accIds { values = append(values, fmt.Sprintf(valuesFormat, - accId, - pubAccountName, pubName, - pubDbName, pubTables, pubComment, + accId, accIdInfoMap[accId].Name, + pubAccountId, pubAccountName, + pubName, pubDbName, pubTables, pubComment, pubsub.SubStatusNormal, )) } @@ -1608,18 +1614,14 @@ func dropSubAccountNameInSubAccounts(ctx context.Context, bh BackgroundExec, pub return bh.Exec(defines.AttachAccountId(ctx, catalog.System_Account), sql) } -func getSqlForInsertIntoMoPubs(ctx context.Context, pubName, databaseName string, databaseId uint64, allTable bool, tableList, accountList string, comment string, checkNameValid bool) (string, error) { +func getSqlForInsertIntoMoPubs(ctx context.Context, accountId uint32, accountName string, pubName, databaseName string, databaseId uint64, allTable bool, tableList, accountList string, comment string, checkNameValid bool) (string, error) { if checkNameValid { if err := inputNameIsInvalid(ctx, pubName, databaseName); err != nil { return "", err } } - accountId, err := defines.GetAccountId(ctx) - if err != nil { - return "", err - } - return fmt.Sprintf(insertIntoMoPubsFormat, accountId, pubName, databaseName, databaseId, allTable, tableList, accountList, defines.GetRoleId(ctx), defines.GetUserId(ctx), comment), nil + return fmt.Sprintf(insertIntoMoPubsFormat, accountId, accountName, pubName, databaseName, databaseId, allTable, tableList, accountList, defines.GetRoleId(ctx), defines.GetUserId(ctx), comment), nil } func getSqlForGetDbIdAndType(ctx context.Context, dbName string, checkNameValid bool, accountId uint64) (string, error) { diff --git a/pkg/frontend/publication_subscription_test.go b/pkg/frontend/publication_subscription_test.go index 0848910e95653..29e78cf149ec7 100644 --- a/pkg/frontend/publication_subscription_test.go +++ b/pkg/frontend/publication_subscription_test.go @@ -133,16 +133,17 @@ func Test_doAlterPublication(t *testing.T) { er := mock_frontend.NewMockExecResult(ctrl) er.EXPECT().GetRowCount().Return(uint64(1)).AnyTimes() er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(0)).Return(int64(0), nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(1)).Return("pub1", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(2)).Return("db1", nil).AnyTimes() - er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(3)).Return(uint64(0), nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(4)).Return("*", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(1)).Return("sys", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(2)).Return("pub1", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(3)).Return("db1", nil).AnyTimes() + er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(4)).Return(uint64(0), nil).AnyTimes() er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(5)).Return("*", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(6)).Return("", nil).AnyTimes() - er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(7)).Return(true, nil).AnyTimes() - er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(8)).Return(uint64(0), nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(6)).Return("*", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(7)).Return("", nil).AnyTimes() + er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(8)).Return(true, nil).AnyTimes() er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(9)).Return(uint64(0), nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(10)).Return("", nil).AnyTimes() + er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(10)).Return(uint64(0), nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(11)).Return("", nil).AnyTimes() return []interface{}{er} } @@ -166,15 +167,17 @@ func Test_doAlterPublication(t *testing.T) { er := mock_frontend.NewMockExecResult(ctrl) er.EXPECT().GetRowCount().Return(uint64(1)).AnyTimes() er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(0)).Return(int64(1), nil).AnyTimes() - er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(1)).Return(true, nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(1)).Return("acc1", nil).AnyTimes() er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(2)).Return(true, nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(3)).Return("sys", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(4)).Return("pub1", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(5)).Return("db1", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(6)).Return("*", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(7)).Return("", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(8)).Return("", nil).AnyTimes() - er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(9)).Return(int64(0), nil).AnyTimes() + er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(3)).Return(true, nil).AnyTimes() + er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(4)).Return(int64(0), nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(5)).Return("sys", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(6)).Return("pub1", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(7)).Return("db1", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(8)).Return("*", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(9)).Return("", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(10)).Return("", nil).AnyTimes() + er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(11)).Return(int64(0), nil).AnyTimes() return []interface{}{er} } @@ -260,16 +263,17 @@ func Test_doAlterPublication2(t *testing.T) { er := mock_frontend.NewMockExecResult(ctrl) er.EXPECT().GetRowCount().Return(uint64(1)).AnyTimes() er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(0)).Return(int64(0), nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(1)).Return("pub1", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(2)).Return("db1", nil).AnyTimes() - er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(3)).Return(uint64(0), nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(4)).Return("*", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(1)).Return("sys", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(2)).Return("pub1", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(3)).Return("db1", nil).AnyTimes() + er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(4)).Return(uint64(0), nil).AnyTimes() er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(5)).Return("*", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(6)).Return("", nil).AnyTimes() - er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(7)).Return(true, nil).AnyTimes() - er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(8)).Return(uint64(0), nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(6)).Return("*", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(7)).Return("", nil).AnyTimes() + er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(8)).Return(true, nil).AnyTimes() er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(9)).Return(uint64(0), nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(10)).Return("", nil).AnyTimes() + er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(10)).Return(uint64(0), nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(11)).Return("", nil).AnyTimes() return []interface{}{er} } @@ -332,37 +336,21 @@ func Test_doAlterPublication2(t *testing.T) { } func Test_doDropPublication(t *testing.T) { - mockedAccountsResults := func(ctrl *gomock.Controller) []interface{} { - er := mock_frontend.NewMockExecResult(ctrl) - er.EXPECT().GetRowCount().Return(uint64(2)).AnyTimes() - er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(0)).Return(int64(0), nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(1)).Return("sys", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(2)).Return("open", nil).AnyTimes() - er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(3)).Return(uint64(1), nil).AnyTimes() - er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(4)).Return(true, nil).AnyTimes() - - er.EXPECT().GetInt64(gomock.Any(), uint64(1), uint64(0)).Return(int64(1), nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(1), uint64(1)).Return("acc1", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(1), uint64(2)).Return("open", nil).AnyTimes() - er.EXPECT().GetUint64(gomock.Any(), uint64(1), uint64(3)).Return(uint64(1), nil).AnyTimes() - er.EXPECT().ColumnIsNull(gomock.Any(), uint64(1), uint64(4)).Return(true, nil).AnyTimes() - return []interface{}{er} - } - mockedPubInfoResults := func(ctrl *gomock.Controller) []interface{} { er := mock_frontend.NewMockExecResult(ctrl) er.EXPECT().GetRowCount().Return(uint64(1)).AnyTimes() er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(0)).Return(int64(0), nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(1)).Return("pub1", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(2)).Return("db1", nil).AnyTimes() - er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(3)).Return(uint64(0), nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(4)).Return("*", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(1)).Return("sys", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(2)).Return("pub1", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(3)).Return("db1", nil).AnyTimes() + er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(4)).Return(uint64(0), nil).AnyTimes() er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(5)).Return("*", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(6)).Return("", nil).AnyTimes() - er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(7)).Return(true, nil).AnyTimes() - er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(8)).Return(uint64(0), nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(6)).Return("*", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(7)).Return("", nil).AnyTimes() + er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(8)).Return(true, nil).AnyTimes() er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(9)).Return(uint64(0), nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(10)).Return("", nil).AnyTimes() + er.EXPECT().GetUint64(gomock.Any(), uint64(0), uint64(10)).Return(uint64(0), nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(11)).Return("", nil).AnyTimes() return []interface{}{er} } @@ -370,15 +358,17 @@ func Test_doDropPublication(t *testing.T) { er := mock_frontend.NewMockExecResult(ctrl) er.EXPECT().GetRowCount().Return(uint64(1)).AnyTimes() er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(0)).Return(int64(1), nil).AnyTimes() - er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(1)).Return(true, nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(1)).Return("acc1", nil).AnyTimes() er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(2)).Return(true, nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(3)).Return("sys", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(4)).Return("pub1", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(5)).Return("db1", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(6)).Return("*", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(7)).Return("", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(8)).Return("", nil).AnyTimes() - er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(9)).Return(int64(0), nil).AnyTimes() + er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(3)).Return(true, nil).AnyTimes() + er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(4)).Return(int64(0), nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(5)).Return("sys", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(6)).Return("pub1", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(7)).Return("db1", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(8)).Return("*", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(9)).Return("", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(10)).Return("", nil).AnyTimes() + er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(11)).Return(int64(0), nil).AnyTimes() return []interface{}{er} } @@ -396,9 +386,6 @@ func Test_doDropPublication(t *testing.T) { bh := mock_frontend.NewMockBackgroundExec(ctrl) bh.EXPECT().Close().Return().AnyTimes() bh.EXPECT().ClearExecResultSet().Return().AnyTimes() - // get all accounts - bh.EXPECT().Exec(gomock.Any(), getAccountIdNamesSql).Return(nil).AnyTimes() - bh.EXPECT().GetExecResultSet().Return(mockedAccountsResults(ctrl)) // get pub info bh.EXPECT().Exec(gomock.Any(), gomock.Any()).Return(nil).AnyTimes() bh.EXPECT().GetExecResultSet().Return(mockedPubInfoResults(ctrl)) @@ -408,7 +395,7 @@ func Test_doDropPublication(t *testing.T) { // deleteMoSubs bh.EXPECT().Exec(gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - err := dropPublication(ctx, bh, true, "pub") + err := dropPublication(ctx, bh, true, "acc1", "pub") convey.So(err, convey.ShouldBeNil) }) } @@ -437,7 +424,7 @@ func TestGetSqlForInsertIntoMoPubs(t *testing.T) { }, } for _, k := range kases { - _, err := getSqlForInsertIntoMoPubs(ctx, k.pubName, k.databaseName, 0, false, "", "", "", true) + _, err := getSqlForInsertIntoMoPubs(ctx, 0, "sys", k.pubName, k.databaseName, 0, false, "", "", "", true) require.Equal(t, k.err, err != nil) } } @@ -499,15 +486,17 @@ func Test_doShowSubscriptions(t *testing.T) { er := mock_frontend.NewMockExecResult(ctrl) er.EXPECT().GetRowCount().Return(uint64(1)).AnyTimes() er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(0)).Return(int64(1), nil).AnyTimes() - er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(1)).Return(true, nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(1)).Return("acc1", nil).AnyTimes() er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(2)).Return(true, nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(3)).Return("sys", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(4)).Return("pub1", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(5)).Return("db1", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(6)).Return("*", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(7)).Return("2024-10-10 11:12:00", nil).AnyTimes() - er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(8)).Return("", nil).AnyTimes() - er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(9)).Return(int64(0), nil).AnyTimes() + er.EXPECT().ColumnIsNull(gomock.Any(), uint64(0), uint64(3)).Return(true, nil).AnyTimes() + er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(4)).Return(int64(0), nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(5)).Return("sys", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(6)).Return("pub1", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(7)).Return("db1", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(8)).Return("*", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(9)).Return("2024-10-10 11:12:00", nil).AnyTimes() + er.EXPECT().GetString(gomock.Any(), uint64(0), uint64(10)).Return("", nil).AnyTimes() + er.EXPECT().GetInt64(gomock.Any(), uint64(0), uint64(11)).Return(int64(0), nil).AnyTimes() return []interface{}{er} } diff --git a/pkg/frontend/snapshot.go b/pkg/frontend/snapshot.go index 967b451b8a3aa..5404d22137221 100644 --- a/pkg/frontend/snapshot.go +++ b/pkg/frontend/snapshot.go @@ -72,7 +72,7 @@ var ( getCurrentExistsAccountsFmt = "select account_id, account_name from mo_catalog.mo_account;" - getSubsSqlFmt = "select sub_account_id, sub_name, sub_time, pub_account_name, pub_name, pub_database, pub_tables, pub_time, pub_comment, status from mo_catalog.mo_subs %s where 1=1" + getSubsSqlFmt = "select sub_account_id, sub_account_name, sub_name, sub_time, pub_account_id, pub_account_name, pub_name, pub_database, pub_tables, pub_time, pub_comment, status from mo_catalog.mo_subs %s where 1=1" checkTableIsMasterFormat = "select db_name, table_name from mo_catalog.mo_foreign_keys where refer_db_name = '%s' and refer_table_name = '%s'" @@ -2310,7 +2310,7 @@ func dropDb(ctx context.Context, bh BackgroundExec, dbName string) (err error) { return err } for _, pubInfo := range pubInfos { - if err = dropPublication(ctx, bh, true, pubInfo.PubName); err != nil { + if err = dropPublication(ctx, bh, true, pubInfo.PubAccountName, pubInfo.PubName); err != nil { return } } diff --git a/pkg/logservice/service_commands.go b/pkg/logservice/service_commands.go index bf0c39cffae90..57697abe7e1c3 100644 --- a/pkg/logservice/service_commands.go +++ b/pkg/logservice/service_commands.go @@ -242,9 +242,6 @@ func (s *Service) heartbeat(ctx context.Context) { s.haClient = cc } - // check the logService TN replica's health on this store. - s.checkReplicaHealth(ctx2) - hb := s.store.getHeartbeatMessage() hb.TaskServiceCreated = s.taskServiceCreated() hb.ConfigData = s.config.GetData() @@ -253,10 +250,15 @@ func (s *Service) heartbeat(ctx context.Context) { if err != nil { err = moerr.AttachCause(ctx2, err) v2.LogHeartbeatFailureCounter.Inc() + v2.LogServiceReplicaHealthGauge.Set(0) s.runtime.Logger().Error("failed to send log service heartbeat", zap.Error(err)) return } + // check the logService TN replica's health on this store. + // NB: do the health checking after heartbeat + s.checkReplicaHealth(ctx2) + s.config.DecrCount() s.handleCommands(cb.Commands) diff --git a/pkg/objectio/ckputil/sinker.go b/pkg/objectio/ckputil/sinker.go new file mode 100644 index 0000000000000..034ba73c9ace3 --- /dev/null +++ b/pkg/objectio/ckputil/sinker.go @@ -0,0 +1,61 @@ +// Copyright 2021 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 ckputil + +import ( + "github.com/matrixorigin/matrixone/pkg/common/mpool" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/fileservice" + "github.com/matrixorigin/matrixone/pkg/objectio" + "github.com/matrixorigin/matrixone/pkg/objectio/ioutil" +) + +func EncodeCluser( + packer *types.Packer, + tableId uint64, + obj *objectio.ObjectId, +) { + packer.EncodeUint64(tableId) + packer.EncodeObjectid(obj) +} + +var SinkerFactory ioutil.FileSinkerFactory + +func init() { + SinkerFactory = ioutil.NewFSinkerImplFactory( + TableObjectsSeqnums, + TableObjectsAttr_Cluster_Idx, + false, + false, + 0, + ) +} + +func NewSinker( + mp *mpool.MPool, + fs fileservice.FileService, + opts ...ioutil.SinkerOption, +) *ioutil.Sinker { + opts = append(opts, ioutil.WithTailSizeCap(0)) + return ioutil.NewSinker( + TableObjectsAttr_Cluster_Idx, + TableObjectsAttrs, + TableObjectsTypes, + SinkerFactory, + mp, + fs, + opts..., + ) +} diff --git a/pkg/objectio/ckputil/sinker_test.go b/pkg/objectio/ckputil/sinker_test.go new file mode 100644 index 0000000000000..9d9c9b9969a09 --- /dev/null +++ b/pkg/objectio/ckputil/sinker_test.go @@ -0,0 +1,173 @@ +// Copyright 2021 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 ckputil + +import ( + "bytes" + "context" + "sort" + "testing" + + "github.com/matrixorigin/matrixone/pkg/common/mpool" + "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/defines" + "github.com/matrixorigin/matrixone/pkg/fileservice" + "github.com/matrixorigin/matrixone/pkg/objectio" + "github.com/matrixorigin/matrixone/pkg/objectio/ioutil" + "github.com/matrixorigin/matrixone/pkg/testutil" + "github.com/stretchr/testify/require" +) + +func Test_ClusterKey1(t *testing.T) { + packer := types.NewPacker() + defer packer.Close() + + tableId := uint64(20) + obj := types.NewObjectid() + + EncodeCluser(packer, tableId, obj) + + buf := packer.Bytes() + packer.Reset() + + tuple, _, schemas, err := types.DecodeTuple(buf) + require.NoError(t, err) + require.Equalf(t, 2, len(schemas), "schemas: %v", schemas) + require.Equalf(t, types.T_uint64, schemas[0], "schemas: %v", schemas) + require.Equalf(t, types.T_Objectid, schemas[1], "schemas: %v", schemas) + + t.Log(tuple.SQLStrings(nil)) + require.Equal(t, tableId, tuple[0].(uint64)) + oid := tuple[1].(types.Objectid) + require.True(t, obj.EQ(&oid)) +} + +func Test_ClusterKey2(t *testing.T) { + packer := types.NewPacker() + defer packer.Close() + cnt := 5000 + clusters := make([][]byte, 0, cnt) + objTemplate := types.NewObjectid() + for i := cnt; i >= 1; i-- { + obj := objTemplate.Copy(uint16(i)) + EncodeCluser(packer, 1, &obj) + clusters = append(clusters, packer.Bytes()) + packer.Reset() + } + sort.Slice(clusters, func(i, j int) bool { + return bytes.Compare(clusters[i], clusters[j]) < 0 + }) + + last := uint16(0) + for _, cluster := range clusters { + tuple, _, _, err := types.DecodeTuple(cluster) + require.NoError(t, err) + require.Equalf(t, 2, len(tuple), "%v", tuple) + require.Equal(t, uint64(1), tuple[0].(uint64)) + obj := tuple[1].(types.Objectid) + curr := obj.Offset() + require.Truef(t, curr > last, "%v,%v", curr, last) + last = curr + } +} + +func Test_Sinker1(t *testing.T) { + proc := testutil.NewProc() + fs, err := fileservice.Get[fileservice.FileService]( + proc.GetFileService(), defines.SharedFileServiceName, + ) + require.NoError(t, err) + mp := proc.Mp() + + bat := NewObjectListBatch() + accountId := uint32(1) + mapping := map[uint64][]uint64{ + 1: {41, 31, 21, 11, 1}, + 2: {42, 32, 22, 12, 2}, + 3: {43, 33, 23, 13, 3}, + } + dbs := []uint64{1, 2, 3} + + sinker := NewSinker( + mp, + fs, + ioutil.WithMemorySizeThreshold(mpool.KB), + ) + defer sinker.Close() + + packer := types.NewPacker() + defer packer.Close() + + fillNext := func(data *batch.Batch, rows int) { + data.CleanOnlyData() + for i, vec := range data.Vecs { + if i == TableObjectsAttr_Accout_Idx { + for j := 0; j < rows; j++ { + require.NoError(t, vector.AppendMultiFixed(vec, accountId, false, rows, mp)) + } + } else if i == TableObjectsAttr_DB_Idx { + tableVec := data.Vecs[TableObjectsAttr_Table_Idx] + idVec := data.Vecs[TableObjectsAttr_ID_Idx] + clusterVec := data.Vecs[TableObjectsAttr_Cluster_Idx] + for j := 0; j < rows; j++ { + dbid := dbs[j%len(dbs)] + tables := mapping[dbid] + tableid := tables[j%len(tables)] + + var obj objectio.ObjectStats + objname := objectio.MockObjectName() + objectio.SetObjectStatsObjectName(&obj, objname) + packer.Reset() + EncodeCluser(packer, tableid, objname.ObjectId()) + + require.NoError(t, vector.AppendFixed(vec, dbid, false, mp)) + require.NoError(t, vector.AppendFixed(tableVec, tableid, false, mp)) + require.NoError(t, vector.AppendBytes(idVec, []byte(objname), false, mp)) + require.NoError(t, vector.AppendBytes(clusterVec, packer.Bytes(), false, mp)) + } + } else if i == TableObjectsAttr_CreateTS_Idx { + for j := 0; j < rows; j++ { + require.NoError(t, vector.AppendFixed(vec, types.NextGlobalTsForTest(), false, mp)) + } + } else if i == TableObjectsAttr_DeleteTS_Idx { + for j := 0; j < rows; j++ { + require.NoError(t, vector.AppendFixed(vec, types.NextGlobalTsForTest(), false, mp)) + } + } + } + data.SetRowCount(rows) + } + + ctx := context.Background() + + rows := 0 + for i := 0; i < 5; i++ { + fillNext(bat, 100) + require.NoError(t, sinker.Write(ctx, bat)) + rows += 100 + } + require.NoError(t, sinker.Sync(ctx)) + files, inMems := sinker.GetResult() + require.Equal(t, 0, len(inMems)) + totalRows := 0 + for _, file := range files { + t.Log(file.String()) + totalRows += int(file.Rows()) + } + require.Equal(t, 5, len(files)) + require.Equal(t, rows, totalRows) +} diff --git a/pkg/objectio/ckputil/types.go b/pkg/objectio/ckputil/types.go index 7b99237151fea..a7e71cc16ed08 100644 --- a/pkg/objectio/ckputil/types.go +++ b/pkg/objectio/ckputil/types.go @@ -13,3 +13,68 @@ // limitations under the License. package ckputil + +import ( + "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/types" +) + +// 1. All checkpoint files should be sorted by `cluster` key. And the +// `cluster` key is `table`+`id`. +// 2. For a specific table, we can save the checkpoint meta like this: +// `table`: table id +// `from`: the start rowid in the checkpoint files +// `to`: the end rowid in the checkpoint files + +const ( + TableObjectsAttr_Accout = "account_id" + TableObjectsAttr_DB = "db_id" + TableObjectsAttr_Table = "table_id" + TableObjectsAttr_ID = "id" + TableObjectsAttr_CreateTS = "create_ts" + TableObjectsAttr_DeleteTS = "delete_ts" + + // TableObjects should be clustered by `table`+`id` + TableObjectsAttr_Cluster = "cluster" + + // TableObjectsAttr_PhysicalAddr = objectio.PhysicalAddr_Attr +) + +const ( + TableObjectsAttr_Accout_Idx = 0 + TableObjectsAttr_DB_Idx = 1 + TableObjectsAttr_Table_Idx = 2 + TableObjectsAttr_ID_Idx = 3 + TableObjectsAttr_CreateTS_Idx = 4 + TableObjectsAttr_DeleteTS_Idx = 5 + TableObjectsAttr_Cluster_Idx = 6 + // TableObjectsAttr_PhysicalAddr_Idx = 7 +) + +var TableObjectsAttrs = []string{ + TableObjectsAttr_Accout, + TableObjectsAttr_DB, + TableObjectsAttr_Table, + TableObjectsAttr_ID, + TableObjectsAttr_CreateTS, + TableObjectsAttr_DeleteTS, + TableObjectsAttr_Cluster, + // TableObjectsAttr_PhysicalAddr, +} + +var TableObjectsTypes = []types.Type{ + types.T_uint32.ToType(), + types.T_uint64.ToType(), + types.T_uint64.ToType(), + types.T_char.ToType(), + types.T_TS.ToType(), + types.T_TS.ToType(), + types.T_char.ToType(), + // types.T_Rowid.ToType(), +} + +var TableObjectsSeqnums = []uint16{0, 1, 2, 3, 4, 5, 6} + +func NewObjectListBatch() *batch.Batch { + return batch.NewWithSchema(false, TableObjectsAttrs, TableObjectsTypes) +} diff --git a/pkg/vm/engine/tae/db/checkpoint/executor.go b/pkg/vm/engine/tae/db/checkpoint/executor.go index 28f7c65588a5c..d3397088c5beb 100644 --- a/pkg/vm/engine/tae/db/checkpoint/executor.go +++ b/pkg/vm/engine/tae/db/checkpoint/executor.go @@ -388,7 +388,6 @@ func (executor *checkpointExecutor) StopWithCause(cause error) { executor.runningICKP.Store(nil) executor.ickpQueue.Stop() executor.gckpQueue.Stop() - executor.runner = nil logutil.Info( "CKP-Executor-Stopped", zap.Error(cause), diff --git a/pkg/vm/engine/tae/rpc/tool_test.go b/pkg/vm/engine/tae/rpc/tool_test.go index 2704e6655b161..68dc541db7f31 100644 --- a/pkg/vm/engine/tae/rpc/tool_test.go +++ b/pkg/vm/engine/tae/rpc/tool_test.go @@ -52,6 +52,7 @@ func Test_gcArg(t *testing.T) { options.WithCheckpointGlobalMinCount(1)(opts) options.WithDisableGCCatalog()(opts) options.WithCheckpointIncrementaInterval(time.Hour)(opts) + options.WithCheckpointGlobalMinCount(100) tae := testutil.NewTestEngine(ctx, ModuleName, t, opts) defer tae.Close() dir := path.Join(tae.Dir, "data") @@ -94,6 +95,9 @@ func Test_gcArg(t *testing.T) { }) assert.NoError(t, txn.Commit(context.Background())) + tae.DB.BGCheckpointRunner.DisableCheckpoint(ctx) + assert.NoError(t, tae.DB.BGCheckpointRunner.WaitRunningCKPDoneForTest(ctx, true)) + assert.NoError(t, tae.DB.BGCheckpointRunner.WaitRunningCKPDoneForTest(ctx, false)) assert.NoError(t, dump.Run()) diff --git a/test/distributed/cases/foreign_key/fk_show_columns.result b/test/distributed/cases/foreign_key/fk_show_columns.result index ecacd9e9776c7..c01ae47fce366 100644 --- a/test/distributed/cases/foreign_key/fk_show_columns.result +++ b/test/distributed/cases/foreign_key/fk_show_columns.result @@ -203,6 +203,7 @@ partition_table_name VARCHAR(1024) NO null desc mo_catalog.mo_pubs ; Field Type Null Key Default Extra Comment account_id INT(32) NO PRI null +account_name VARCHAR(300) YES null pub_name VARCHAR(64) NO PRI null database_name VARCHAR(5000) YES null database_id BIGINT UNSIGNED(64) YES null @@ -526,6 +527,7 @@ partition_table_name VARCHAR(1024) NO null show columns from mo_catalog.mo_pubs ; Field Type Null Key Default Extra Comment account_id INT(32) NO PRI null +account_name VARCHAR(300) YES null pub_name VARCHAR(64) NO PRI null database_name VARCHAR(5000) YES null database_id BIGINT UNSIGNED(64) YES null diff --git a/test/distributed/cases/publication_subscription/pub_sub3.result b/test/distributed/cases/publication_subscription/pub_sub3.result index 286abb34d3e8a..37c463487aafb 100644 --- a/test/distributed/cases/publication_subscription/pub_sub3.result +++ b/test/distributed/cases/publication_subscription/pub_sub3.result @@ -69,11 +69,11 @@ create publication pub03 database db02 table t1, t2 account acc02 comment 'publi drop database if exists sub02; create database sub02 from acc01 publication pub01; select * from mo_catalog.mo_subs; -sub_account_id sub_name sub_time pub_account_name pub_name pub_database pub_tables pub_time pub_comment status -67 null null acc01 pub02 db02 * 2024-08-14 18:34:52 publish to acc01 and acc03 0 -66 null null acc01 pub02 db02 * 2024-08-14 18:34:52 publish to acc01 and acc03 0 -66 null null acc01 pub03 db02 t1,t2 2024-08-14 18:34:52 publish to acc02 0 -66 sub02 2024-08-14 18:34:52 acc01 pub01 db02 * 2024-08-14 18:34:52 publish to acc01 0 +sub_account_id sub_account_name sub_name sub_time pub_account_id pub_account_name pub_name pub_database pub_tables pub_time pub_comment status +4 acc02 null null 3 acc01 pub02 db02 * 2024-12-31 18:14:15 publish to acc01 and acc03 0 +5 acc03 null null 3 acc01 pub02 db02 * 2024-12-31 18:14:15 publish to acc01 and acc03 0 +4 acc02 null null 3 acc01 pub03 db02 t1,t2 2024-12-31 18:14:15 publish to acc02 0 +4 acc02 sub02 2024-12-31 18:14:15 3 acc01 pub01 db02 * 2024-12-31 18:14:15 publish to acc01 0 drop database sub02; drop publication pub01; drop publication pub02; diff --git a/test/distributed/cases/publication_subscription/pub_sub3.sql b/test/distributed/cases/publication_subscription/pub_sub3.sql index 8241e2521b31f..1fd3fca27044e 100644 --- a/test/distributed/cases/publication_subscription/pub_sub3.sql +++ b/test/distributed/cases/publication_subscription/pub_sub3.sql @@ -81,7 +81,7 @@ drop database if exists sub02; create database sub02 from acc01 publication pub01; -- @session --- @ignore:0,2,7 +-- @ignore:0,3,4,9 select * from mo_catalog.mo_subs; -- @session:id=2&user=acc02:test_account&password=111 diff --git a/test/distributed/cases/snapshot/cluster/restore_cluster_table.result b/test/distributed/cases/snapshot/cluster/restore_cluster_table.result index 123dc6b1539f9..1896a91e93bee 100644 --- a/test/distributed/cases/snapshot/cluster/restore_cluster_table.result +++ b/test/distributed/cases/snapshot/cluster/restore_cluster_table.result @@ -222,8 +222,8 @@ id val 3 c select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -1 test_if_hit_elseif_first_elseif null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:44:26 2024-08-07 09:44:26 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci -2 test_if_hit_if null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:44:26 2024-08-07 09:44:26 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +1 test_if_hit_elseif_first_elseif null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-12-31 06:24:14 2024-12-31 06:24:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +2 test_if_hit_if null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-12-31 06:24:14 2024-12-31 06:24:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop database if exists udf_db2; create database udf_db2; use udf_db2; @@ -232,15 +232,15 @@ language sql as '$1 + $2'; select * from mo_catalog.mo_user_defined_function; function_id name owner args rettype body language db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -1 addab 2 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db2 test_account 2024-08-07 09:44:26 2024-08-07 09:44:26 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +1 addab 2 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db2 test_account 2024-12-31 06:24:14 2024-12-31 06:24:14 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop stage if exists my_ext_stage; create stage my_ext_stage URL='s3://load/files/'; drop stage if exists my_ext_stage1; create stage my_ext_stage1 URL='s3://load/files/' CREDENTIALS={'AWS_KEY_ID'='1a2b3c' ,'AWS_SECRET_KEY'='4x5y6z'}; select * from mo_catalog.mo_stages; stage_id stage_name url stage_credentials stage_status created_time comment -1 my_ext_stage s3://load/files/ disabled 2024-08-07 09:44:26 -2 my_ext_stage1 s3://load/files/ AWS_KEY_ID=1a2b3c,AWS_SECRET_KEY=4x5y6z disabled 2024-08-07 09:44:26 +1 my_ext_stage s3://load/files/ disabled 2024-12-31 06:24:14 +2 my_ext_stage1 s3://load/files/ AWS_KEY_ID=1a2b3c,AWS_SECRET_KEY=4x5y6z disabled 2024-12-31 06:24:14 drop database if exists test01; drop database if exists test02; drop database if exists test03; @@ -376,7 +376,7 @@ drop snapshot if exists cluster_level_snapshot; create snapshot cluster_level_snapshot for cluster; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -cluster_level_snapshot 2024-08-07 09:44:26.651225093 cluster +cluster_level_snapshot 2024-12-31 06:24:16.087227 cluster use db02; alter table departments add column newcolumn int after department_id; show create table departments; @@ -454,14 +454,14 @@ drop function udf_db2.`addAB`(x int, y int); drop stage if exists my_ext_stage; select * from mo_catalog.mo_stages; stage_id stage_name url stage_credentials stage_status created_time comment -2 my_ext_stage1 s3://load/files/ AWS_KEY_ID=1a2b3c,AWS_SECRET_KEY=4x5y6z disabled 2024-08-07 09:44:26 +2 my_ext_stage1 s3://load/files/ AWS_KEY_ID=1a2b3c,AWS_SECRET_KEY=4x5y6z disabled 2024-12-31 06:24:14 use udf_db2; create function `add`(x int, y int) returns int language sql as '$1 + $2'; select * from mo_catalog.mo_user_defined_function; function_id name owner args rettype body language db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -2 add 2 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db2 test_account 2024-08-07 09:44:26 2024-08-07 09:44:26 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +2 add 2 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db2 test_account 2024-12-31 06:24:16 2024-12-31 06:24:16 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop database test01; use test02; select * from v01; @@ -557,15 +557,15 @@ select * from repub01.pri01; Unknown database repub01 select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -1 test_if_hit_elseif_first_elseif null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:44:26 2024-08-07 09:44:26 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci -2 test_if_hit_if null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:44:26 2024-08-07 09:44:26 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +1 test_if_hit_elseif_first_elseif null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-12-31 06:24:14 2024-12-31 06:24:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +2 test_if_hit_if null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-12-31 06:24:14 2024-12-31 06:24:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci select * from mo_catalog.mo_stages; stage_id stage_name url stage_credentials stage_status created_time comment -1 my_ext_stage s3://load/files/ disabled 2024-08-07 09:44:26 -2 my_ext_stage1 s3://load/files/ AWS_KEY_ID=1a2b3c,AWS_SECRET_KEY=4x5y6z disabled 2024-08-07 09:44:26 +1 my_ext_stage s3://load/files/ disabled 2024-12-31 06:24:14 +2 my_ext_stage1 s3://load/files/ AWS_KEY_ID=1a2b3c,AWS_SECRET_KEY=4x5y6z disabled 2024-12-31 06:24:14 select * from mo_catalog.mo_user_defined_function; function_id name owner args rettype body language db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -1 addab 2 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db2 test_account 2024-07-10 07:43:56 2024-07-10 07:43:56 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +1 addab 2 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db2 test_account 2024-12-31 06:24:14 2024-12-31 06:24:14 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci select * from test01.sales; id sale_date amount 1 2018-12-25 100.00 @@ -600,8 +600,8 @@ drop database test; drop database repub02; select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -1 test_if_hit_elseif_first_elseif null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:44:26 2024-08-07 09:44:26 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci -2 test_if_hit_if null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:44:26 2024-08-07 09:44:26 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +1 test_if_hit_elseif_first_elseif null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-12-31 06:24:14 2024-12-31 06:24:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +2 test_if_hit_if null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-12-31 06:24:14 2024-12-31 06:24:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop database if exists udf_db2; drop function `addab`(x int, y int); function addab doesn't exist @@ -631,7 +631,7 @@ drop snapshot if exists cluster_level_snapshot; create snapshot cluster_level_snapshot for cluster; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -cluster_level_snapshot 2024-08-07 09:44:29.17621458 cluster +cluster_level_snapshot 2024-12-31 06:24:33.682324 cluster use mo_catalog; drop table if exists t2; create cluster table t2(a int); @@ -656,7 +656,7 @@ drop snapshot if exists cluster_level_snapshot; create snapshot cluster_level_snapshot for cluster; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -cluster_level_snapshot 2024-08-07 09:44:29.441979616 cluster +cluster_level_snapshot 2024-12-31 06:24:34.106923 cluster use mo_catalog; drop table if exists t2; create cluster table t2(a int); @@ -678,10 +678,10 @@ drop table if exists t1; create cluster table t1(a int); insert into t1 values (1,6),(2,6),(3,6); drop snapshot if exists account_level_snapshot; -create snapshot account_level_snapshot for account sys; +create snapshot account_level_snapshot for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -account_level_snapshot 2024-08-07 09:44:29.684132745 account sys +account_level_snapshot 2024-12-31 06:24:34.744824 account sys use mo_catalog; drop table if exists t2; create cluster table t2(a int); diff --git a/test/distributed/cases/snapshot/cluster/restore_cluster_table.sql b/test/distributed/cases/snapshot/cluster/restore_cluster_table.sql index 47f2ff484dfd6..554b605bb1dbb 100644 --- a/test/distributed/cases/snapshot/cluster/restore_cluster_table.sql +++ b/test/distributed/cases/snapshot/cluster/restore_cluster_table.sql @@ -539,7 +539,7 @@ create cluster table t1(a int); insert into t1 values (1,6),(2,6),(3,6); drop snapshot if exists account_level_snapshot; -create snapshot account_level_snapshot for account sys; +create snapshot account_level_snapshot for account; -- @ignore:1 show snapshots; diff --git a/test/distributed/cases/snapshot/cluster/restore_fk_restore_master_table.result b/test/distributed/cases/snapshot/cluster/restore_fk_restore_master_table.result index 0c11c6880cbfe..ffd284df8a09b 100644 --- a/test/distributed/cases/snapshot/cluster/restore_fk_restore_master_table.result +++ b/test/distributed/cases/snapshot/cluster/restore_fk_restore_master_table.result @@ -179,10 +179,10 @@ performancereviewid employeeid reviewdate reviewerid drop snapshot if exists sys_sp; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -create snapshot sys_sp for account sys; +create snapshot sys_sp for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sys_sp 2024-08-20 03:03:00.682696 account sys +sys_sp 2024-12-31 06:16:14.042907 account sys restore account sys database Company from snapshot sys_sp; select * from Departments; departmentid name managerid diff --git a/test/distributed/cases/snapshot/cluster/restore_fk_restore_master_table.sql b/test/distributed/cases/snapshot/cluster/restore_fk_restore_master_table.sql index 1c6866ae107f8..183da02420a3d 100644 --- a/test/distributed/cases/snapshot/cluster/restore_fk_restore_master_table.sql +++ b/test/distributed/cases/snapshot/cluster/restore_fk_restore_master_table.sql @@ -142,7 +142,7 @@ select * from PerformanceReviews; drop snapshot if exists sys_sp; -- @ignore:1 show snapshots; -create snapshot sys_sp for account sys; +create snapshot sys_sp for account; -- @ignore:1 show snapshots; diff --git a/test/distributed/cases/snapshot/cluster/restore_simple_read.result b/test/distributed/cases/snapshot/cluster/restore_simple_read.result index 669b3a40cfa47..f1a7227079fc7 100644 --- a/test/distributed/cases/snapshot/cluster/restore_simple_read.result +++ b/test/distributed/cases/snapshot/cluster/restore_simple_read.result @@ -10,7 +10,7 @@ show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME account_sp 2024-06-28 06:13:04.389609 account sys create snapshot cluster_sp for cluster; -create snapshot account_sp for account sys; +create snapshot account_sp for account; internal error: snapshot account_sp already exists show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME diff --git a/test/distributed/cases/snapshot/cluster/restore_simple_read.sql b/test/distributed/cases/snapshot/cluster/restore_simple_read.sql index c4af921bd9a9e..54237bdb90632 100644 --- a/test/distributed/cases/snapshot/cluster/restore_simple_read.sql +++ b/test/distributed/cases/snapshot/cluster/restore_simple_read.sql @@ -10,7 +10,7 @@ drop snapshot if exists cluster_sp; -- @ignore:1 show snapshots; create snapshot cluster_sp for cluster; -create snapshot account_sp for account sys; +create snapshot account_sp for account; -- @ignore:1 show snapshots; diff --git a/test/distributed/cases/snapshot/cluster/snapshot_restore_skip_master_db.result b/test/distributed/cases/snapshot/cluster/snapshot_restore_skip_master_db.result index 09d89485f9fe1..a09eada1382f8 100644 --- a/test/distributed/cases/snapshot/cluster/snapshot_restore_skip_master_db.result +++ b/test/distributed/cases/snapshot/cluster/snapshot_restore_skip_master_db.result @@ -2,7 +2,7 @@ drop database if exists test03; create database test03; use test03; drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; drop table if exists pri01; create table pri01( deptno int unsigned comment '部门编号', diff --git a/test/distributed/cases/snapshot/cluster/snapshot_restore_skip_master_db.sql b/test/distributed/cases/snapshot/cluster/snapshot_restore_skip_master_db.sql index 24898f9906fd2..92bf7bca67692 100644 --- a/test/distributed/cases/snapshot/cluster/snapshot_restore_skip_master_db.sql +++ b/test/distributed/cases/snapshot/cluster/snapshot_restore_skip_master_db.sql @@ -3,7 +3,7 @@ create database test03; use test03; drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; drop table if exists pri01; create table pri01( diff --git a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_cluster.sql b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_cluster.sql index 45c4144982672..f250702ad7707 100644 --- a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_cluster.sql +++ b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_cluster.sql @@ -199,7 +199,7 @@ drop publication if exists pub02; create publication pub02 database repub02 account acc02 comment 'publish before creating snapshot'; -- @ignore:5,6 show publications; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; drop database if exists procedure_test; diff --git a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_pubsub.result b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_pubsub.result index 309a89b1d2126..040bcc292f5d7 100644 --- a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_pubsub.result +++ b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_pubsub.result @@ -15,13 +15,13 @@ show publications; publication database tables sub_account subscribed_accounts create_time update_time comments publication01 republication01 * acc01 2024-11-14 16:28:52 null publish before creating snapshot select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment -0 publication01 republication01 273171 true * acc01 2024-11-14 16:28:52 null 0 1 publish before creating snapshot +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +0 sys publication01 republication01 272993 true * acc01 2024-12-31 18:15:27 null 0 1 publish before creating snapshot drop publication publication01; drop database republication01; restore account sys from snapshot sp01; select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment use republication01; select * from repub01; col1 @@ -72,8 +72,8 @@ show publications; publication database tables sub_account subscribed_accounts create_time update_time comments pub02 repub02 * acc01 2024-11-14 16:28:52 null publish before creating snapshot select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment -0 pub02 repub02 273197 true * acc01 2024-11-14 16:28:52 null 0 1 publish before creating snapshot +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +0 sys pub02 repub02 273019 true * acc01 2024-12-31 18:15:27 null 0 1 publish before creating snapshot drop database if exists sub01; create database sub01 from sys publication pub02; show databases; @@ -106,7 +106,7 @@ restore account sys from snapshot sp02; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment use repub02; select * from pri01; deptno dname loc @@ -208,8 +208,8 @@ system system_metrics drop snapshot sp01; select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment -0 pub03 repub03 273225 true * acc01 2024-11-14 16:28:53 null 0 1 create repub03 +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +0 sys pub03 repub03 273048 true * acc01 2024-12-31 18:15:28 null 0 1 create repub03 show publications; publication database tables sub_account subscribed_accounts create_time update_time comments pub03 repub03 * acc01 2024-11-14 16:28:53 null create repub03 @@ -230,8 +230,8 @@ create snapshot sp02 for cluster; drop publication if exists pub04; create publication pub04 database db01 account acc01 comment 'create pub04'; select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment -0 pub04 db01 273243 true * acc01 2024-11-14 16:28:53 null 0 1 create pub04 +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +0 sys pub04 db01 273066 true * acc01 2024-12-31 18:15:28 null 0 1 create pub04 show publications; publication database tables sub_account subscribed_accounts create_time update_time comments pub04 db01 * acc01 2024-11-14 16:28:53 null create pub04 @@ -458,8 +458,8 @@ create snapshot sp100 for cluster; drop publication if exists pub07; create publication pub07 database test01 account acc01 comment 'publish test01'; select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment -0 pub07 test01 273400 true * acc01 2024-11-14 16:28:55 null 0 1 publish test01 +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +0 sys pub07 test01 273223 true * acc01 2024-12-31 18:15:31 null 0 1 publish test01 drop snapshot if exists sp101; create snapshot sp101 for cluster; drop database if exists sub07; @@ -481,7 +481,7 @@ use sub07; internal error: there is no publication pub07 restore account sys from snapshot sp101; select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment show databases; Database information_schema diff --git a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_pubsub.sql b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_pubsub.sql index 21feca626a49d..c3ad3136248d2 100644 --- a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_pubsub.sql +++ b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_pubsub.sql @@ -17,13 +17,13 @@ drop publication if exists publication01; create publication publication01 database republication01 account acc01 comment 'publish before creating snapshot'; -- @ignore:5,6 show publications; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; drop publication publication01; drop database republication01; restore account sys from snapshot sp01; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; use republication01; select * from repub01; @@ -81,7 +81,7 @@ drop publication if exists pub02; create publication pub02 database repub02 account acc01 comment 'publish before creating snapshot'; -- @ignore:5,6 show publications; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @session:id=1&user=acc01:test_account&password=111 @@ -97,7 +97,7 @@ select * from aff01; restore account sys from snapshot sp02; -- @ignore:5,6 show publications; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; use repub02; select * from pri01; @@ -163,7 +163,7 @@ show subscriptions; show databases; -- @session drop snapshot sp01; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @ignore:5,6 show publications; @@ -190,7 +190,7 @@ create snapshot sp02 for cluster; drop publication if exists pub04; create publication pub04 database db01 account acc01 comment 'create pub04'; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @ignore:5,6 show publications; @@ -419,7 +419,7 @@ create snapshot sp100 for cluster; drop publication if exists pub07; create publication pub07 database test01 account acc01 comment 'publish test01'; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; drop snapshot if exists sp101; @@ -440,7 +440,7 @@ use sub07; -- @session restore account sys from snapshot sp101; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @session:id=1&user=acc01:test_account&password=111 @@ -605,7 +605,7 @@ create snapshot sp105 for cluster; -- @session:id=1&user=acc01:test_account&password=111 drop publication if exists pub10; create publication pub10 database test04 account acc02 comment 'publish test03'; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @session diff --git a/test/distributed/cases/snapshot/restore_fk_table.result b/test/distributed/cases/snapshot/restore_fk_table.result index 4894e60f5c553..7db5c462fb068 100644 --- a/test/distributed/cases/snapshot/restore_fk_table.result +++ b/test/distributed/cases/snapshot/restore_fk_table.result @@ -59,7 +59,7 @@ create table table04(a INT primary key AUTO_INCREMENT, b INT, c INT); insert into table03 values (1,1,1), (2,2,2); insert into table04 values (0,1,2), (2,3,4); drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; use test01; drop table aff01; drop table pri01; @@ -279,7 +279,7 @@ insert into aff01 values (7900,'JAMES','CLERK',7698,'1981-12-03',950,NULL,30); insert into aff01 values (7902,'FORD','ANALYST',7566,'1981-12-03',3000,NULL,20); insert into aff01 values (7934,'MILLER','CLERK',7782,'1982-01-23',1300,NULL,10); drop snapshot if exists sp03; -create snapshot sp03 for account sys; +create snapshot sp03 for account; drop database test02; drop database test01; restore account sys database test02 table aff01 from snapshot sp03; @@ -311,7 +311,7 @@ insert into f1 values (2,2); insert into c1 values (2,3); insert into c1 values (2,2); drop snapshot if exists sp04; -create snapshot sp04 for account sys; +create snapshot sp04 for account; insert into f1 values (3,20); insert into f1 values (4,600); restore account sys database test04 table f1 from snapshot sp04; @@ -432,7 +432,7 @@ col1 col2 col3 1 1 1 2 2 2 drop snapshot if exists sp07; -create snapshot sp07 for account sys; +create snapshot sp07 for account; insert into foreign01 values(3, '323214321321', 32, 1); insert into foreign02 values(3,2,10); restore account sys database test07 table foreign02 from snapshot sp07; diff --git a/test/distributed/cases/snapshot/restore_fk_table.sql b/test/distributed/cases/snapshot/restore_fk_table.sql index a14f2f82caf06..de005f7a602b2 100644 --- a/test/distributed/cases/snapshot/restore_fk_table.sql +++ b/test/distributed/cases/snapshot/restore_fk_table.sql @@ -67,7 +67,7 @@ insert into table03 values (1,1,1), (2,2,2); insert into table04 values (0,1,2), (2,3,4); drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; use test01; drop table aff01; @@ -240,7 +240,7 @@ insert into aff01 values (7902,'FORD','ANALYST',7566,'1981-12-03',3000,NULL,20); insert into aff01 values (7934,'MILLER','CLERK',7782,'1982-01-23',1300,NULL,10); drop snapshot if exists sp03; -create snapshot sp03 for account sys; +create snapshot sp03 for account; drop database test02; drop database test01; @@ -271,7 +271,7 @@ insert into c1 values (2,3); insert into c1 values (2,2); drop snapshot if exists sp04; -create snapshot sp04 for account sys; +create snapshot sp04 for account; insert into f1 values (3,20); insert into f1 values (4,600); @@ -391,7 +391,7 @@ select * from foreign01; select * from foreign02; drop snapshot if exists sp07; -create snapshot sp07 for account sys; +create snapshot sp07 for account; insert into foreign01 values(3, '323214321321', 32, 1); insert into foreign02 values(3,2,10); diff --git a/test/distributed/cases/snapshot/restore_pub_sub.result b/test/distributed/cases/snapshot/restore_pub_sub.result index 2faf4d9237f30..7bea1ef6295a4 100644 --- a/test/distributed/cases/snapshot/restore_pub_sub.result +++ b/test/distributed/cases/snapshot/restore_pub_sub.result @@ -46,17 +46,17 @@ drop publication if exists pub01; create publication pub01 database test01 account all; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub01 test01 * * 2024-12-17 15:51:00 null +pub01 test01 * * 2024-12-31 11:13:23 null show subscriptions all; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub01 sys test01 * 2024-12-17 15:51:00 null null 0 +pub01 sys test01 * 2024-12-31 11:13:23 null null 0 drop database if exists sub01; create database sub01 from sys publication pub01; drop snapshot if exists sp01; create snapshot sp01 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sp01 2024-12-17 07:51:01.124788 account acc01 +sp01 2024-12-31 03:13:23.222907 account acc01 use sub01; show tables; Tables_in_sub01 @@ -80,7 +80,7 @@ pub_name pub_account pub_database pub_tables pub_comment pub_time restore account acc01 from snapshot sp01; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub01 sys test01 * 2024-12-17 15:51:00 sub01 2024-12-17 15:51:01 0 +pub01 sys test01 * 2024-12-31 11:13:23 sub01 2024-12-31 11:13:23 0 show databases; Database information_schema @@ -96,10 +96,10 @@ use test01; drop table aff01; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub01 test01 * * acc01 2024-12-17 15:51:00 null +pub01 test01 * * acc01 2024-12-31 11:13:23 null show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub01 sys test01 * 2024-12-17 15:51:00 sub01 2024-12-17 15:51:01 0 +pub01 sys test01 * 2024-12-31 11:13:23 sub01 2024-12-31 11:13:23 0 show databases; Database information_schema @@ -153,20 +153,20 @@ drop publication if exists pub02; create publication pub02 database test02 table rs02 account acc01; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub02 test02 rs02 acc01 2024-12-17 15:51:02 null +pub02 test02 rs02 acc01 2024-12-31 11:13:24 null show subscriptions all; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub02 sys test02 rs02 2024-12-17 15:51:02 null null 0 +pub02 sys test02 rs02 2024-12-31 11:13:24 null null 0 drop database if exists sub02; create database sub02 from sys publication pub02; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub02 sys test02 rs02 2024-12-17 15:51:02 sub02 2024-12-17 15:51:02 0 +pub02 sys test02 rs02 2024-12-31 11:13:24 sub02 2024-12-31 11:13:24 0 drop snapshot if exists sp02; create snapshot sp02 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sp02 2024-12-17 07:51:02.40092 account acc01 +sp02 2024-12-31 03:13:24.541639 account acc01 drop database sub02; drop publication pub02; show subscriptions; @@ -210,7 +210,7 @@ drop publication if exists pub03; create publication pub03 database test03 account acc01; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub03 test03 * acc01 2024-12-17 15:51:03 null +pub03 test03 * acc01 2024-12-31 11:13:25 null show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status drop database if exists sub03; @@ -276,14 +276,14 @@ drop publication if exists pub04; create publication pub04 database test04 account sys comment 'pub to sys account'; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub04 test04 * sys 2024-12-17 15:51:04 null pub to sys account +pub04 test04 * sys 2024-12-31 11:13:30 null pub to sys account drop database if exists sub05; create database sub05 from acc01 publication pub04; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub04 acc01 test04 * pub to sys account 2024-12-17 15:51:04 sub05 2024-12-17 15:51:04 0 +pub04 acc01 test04 * pub to sys account 2024-12-31 11:13:30 sub05 2024-12-31 11:13:31 0 drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; drop database sub05; restore account sys database sub05 table t4 from snapshot sp05; internal error: can't restore to table for sub db @@ -316,7 +316,7 @@ count(*) 1 show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub04 acc01 test04 * pub to sys account 2024-12-17 15:51:04 sub05 2024-12-17 15:51:05 0 +pub04 acc01 test04 * pub to sys account 2024-12-31 11:13:30 sub05 2024-12-31 11:13:31 0 drop database sub05; drop snapshot sp05; drop publication pub04; @@ -341,25 +341,25 @@ drop publication if exists pub06; create publication pub06 database test06 account acc02 comment 'acc01 pub to acc02'; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub06 test06 * acc02 2024-12-17 15:51:05 null acc01 pub to acc02 +pub06 test06 * acc02 2024-12-31 11:13:32 null acc01 pub to acc02 drop database if exists sub06; create database sub06 from acc01 publication pub06; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub06 acc01 test06 * acc01 pub to acc02 2024-12-17 15:51:05 sub06 2024-12-17 15:51:05 0 +pub06 acc01 test06 * acc01 pub to acc02 2024-12-31 11:13:32 sub06 2024-12-31 11:13:33 0 restore account acc01 from snapshot sp06; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub06 acc01 null null null null sub06 2024-12-17 15:51:05 2 +pub06 acc01 null null null null sub06 2024-12-31 11:13:33 2 use sub06; internal error: there is no publication pub06 drop publication if exists pub06; create publication pub06 database test06 table table01 account acc02 comment 'acc01 pub to acc02'; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub06 test06 table01 acc02 acc02 2024-12-17 15:51:15 null acc01 pub to acc02 +pub06 test06 table01 acc02 acc02 2024-12-31 11:13:34 null acc01 pub to acc02 show databases; Database information_schema @@ -420,17 +420,17 @@ drop database if exists sub06; create database sub06 from sys publication pub06; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub06 sys test07 * 2024-12-17 15:51:17 sub06 2024-12-17 15:51:17 0 +pub06 sys test07 * 2024-12-31 11:13:34 sub06 2024-12-31 11:13:35 0 drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; alter publication pub06 account acc02 database test07; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub06 sys null null null null sub06 2024-12-17 15:51:17 1 +pub06 sys null null null null sub06 2024-12-31 11:13:35 1 restore account sys from snapshot sp06; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub06 sys null null null null sub06 2024-12-17 15:51:17 2 +pub06 sys null null null null sub06 2024-12-31 11:13:35 2 drop database sub06; drop publication pub06; internal error: publication 'pub06' does not exist @@ -453,7 +453,7 @@ insert into table01 values (2, -1, null); drop publication if exists pub10; create publication pub10 database test01 table table01 account acc01; drop snapshot if exists sp10; -create snapshot sp10 for account sys; +create snapshot sp10 for account; drop table if exists table02; create table table02 (col1 int, col3 decimal); insert into table02 values(1, 2); @@ -479,11 +479,11 @@ system_metrics test01 show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub10 sys null null null null sub01 2024-12-17 15:51:19 2 +pub10 sys null null null null sub01 2024-12-31 11:13:36 2 drop database sub01; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub11 sys null null null null sub02 2024-12-17 15:51:19 2 +pub11 sys null null null null sub02 2024-12-31 11:13:36 2 drop database sub02; drop publication pub10; internal error: publication 'pub10' does not exist @@ -502,26 +502,26 @@ drop publication if exists pub_full01; create publication pub_full01 database pub_fulltext_table account acc01; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub_full01 pub_fulltext_table * acc01 2024-12-17 15:51:19 null +pub_full01 pub_fulltext_table * acc01 2024-12-31 11:13:41 null show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status drop database if exists sub_pull01; create database sub_pull01 from sys publication pub_full01; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub_full01 sys pub_fulltext_table * 2024-12-17 15:51:19 sub_pull01 2024-12-17 15:51:20 0 +pub_full01 sys pub_fulltext_table * 2024-12-31 11:13:41 sub_pull01 2024-12-31 11:13:41 0 drop snapshot if exists sp_pub01; -create snapshot sp_pub01 for account sys; +create snapshot sp_pub01 for account; drop publication pub_full01; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub_full01 sys null null null null sub_pull01 2024-12-17 15:51:20 2 +pub_full01 sys null null null null sub_pull01 2024-12-31 11:13:41 2 restore account sys database pub_fulltext_table from snapshot sp_pub01; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub_full01 sys null null null null sub_pull01 2024-12-17 15:51:20 2 +pub_full01 sys null null null null sub_pull01 2024-12-31 11:13:41 2 use sub_pull01; internal error: there is no publication pub_full01 select * from src; @@ -543,15 +543,15 @@ drop database if exists fulltext_sub02; create database fulltext_sub02 from sys publication pub_full02; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub_full02 sys restore_fulltext_table * 2024-12-17 15:51:20 fulltext_sub02 2024-12-17 15:51:20 0 -pub_full01 sys null null null null sub_pull01 2024-12-17 15:51:20 2 +pub_full02 sys restore_fulltext_table * 2024-12-31 11:13:42 fulltext_sub02 2024-12-31 11:13:42 0 +pub_full01 sys null null null null sub_pull01 2024-12-31 11:13:41 2 drop snapshot if exists sp_full02; create snapshot sp_full02 for account acc01; drop publication pub_full02; drop database fulltext_sub02; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub_full01 sys null null null null sub_pull01 2024-12-17 15:51:20 2 +pub_full01 sys null null null null sub_pull01 2024-12-31 11:13:41 2 restore account acc01 from snapshot sp_full02; show databases; Database diff --git a/test/distributed/cases/snapshot/restore_pub_sub.sql b/test/distributed/cases/snapshot/restore_pub_sub.sql index e386fdd41b2b6..a3226d4e230b1 100644 --- a/test/distributed/cases/snapshot/restore_pub_sub.sql +++ b/test/distributed/cases/snapshot/restore_pub_sub.sql @@ -267,7 +267,7 @@ create database sub05 from acc01 publication pub04; -- @ignore:5,7 show subscriptions; drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; drop database sub05; restore account sys database sub05 table t4 from snapshot sp05; @@ -392,7 +392,7 @@ show subscriptions; -- @session drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; alter publication pub06 account acc02 database test07; @@ -437,7 +437,7 @@ drop publication if exists pub10; create publication pub10 database test01 table table01 account acc01; drop snapshot if exists sp10; -create snapshot sp10 for account sys; +create snapshot sp10 for account; drop table if exists table02; create table table02 (col1 int, col3 decimal); @@ -507,7 +507,7 @@ show subscriptions; -- @session drop snapshot if exists sp_pub01; -create snapshot sp_pub01 for account sys; +create snapshot sp_pub01 for account; drop publication pub_full01; diff --git a/test/distributed/cases/snapshot/restote_sub.result b/test/distributed/cases/snapshot/restore_sub.result similarity index 89% rename from test/distributed/cases/snapshot/restote_sub.result rename to test/distributed/cases/snapshot/restore_sub.result index 7b131cdac3164..08df92b90fa1d 100644 --- a/test/distributed/cases/snapshot/restote_sub.result +++ b/test/distributed/cases/snapshot/restore_sub.result @@ -42,17 +42,17 @@ drop PUBLICATION if exists pub01; create PUBLICATION pub01 database test01 account all; show PUBLICATIONS; publication database tables sub_account subscribed_accounts create_time update_time comments -pub01 test01 * * 2024-11-15 11:34:07 null +pub01 test01 * * 2024-12-31 11:53:39 null show subscriptions all; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub01 sys test01 * 2024-11-15 11:34:07 null null 0 +pub01 sys test01 * 2024-12-31 11:53:39 null null 0 drop database if exists sub01; create database sub01 FROM sys PUBLICATION pub01; drop snapshot if exists sp01; create snapshot sp01 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sp01 2024-11-15 03:34:07.569057 account acc01 +sp01 2024-12-31 03:53:39.574411 account acc01 use sub01; show tables; Tables_in_sub01 @@ -76,15 +76,15 @@ pub_name pub_account pub_database pub_tables pub_comment pub_time restore account acc01 from snapshot sp01; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub01 sys test01 * 2024-11-15 11:34:07 sub01 2024-11-15 11:34:07 0 +pub01 sys test01 * 2024-12-31 11:53:39 sub01 2024-12-31 11:53:39 0 use test01; drop table aff01; show PUBLICATIONs; publication database tables sub_account subscribed_accounts create_time update_time comments -pub01 test01 * * acc01 2024-11-15 11:34:07 null +pub01 test01 * * acc01 2024-12-31 11:53:39 null show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub01 sys test01 * 2024-11-15 11:34:07 sub01 2024-11-15 11:34:07 0 +pub01 sys test01 * 2024-12-31 11:53:39 sub01 2024-12-31 11:53:39 0 show databases; Database information_schema @@ -138,20 +138,20 @@ drop PUBLICATION if exists pub02; create PUBLICATION pub02 database test02 table rs02 account acc01; show PUBLICATIONS; publication database tables sub_account subscribed_accounts create_time update_time comments -pub02 test02 rs02 acc01 2024-11-15 11:34:08 null +pub02 test02 rs02 acc01 2024-12-31 11:53:40 null show subscriptions all; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub02 sys test02 rs02 2024-11-15 11:34:08 null null 0 +pub02 sys test02 rs02 2024-12-31 11:53:40 null null 0 drop database if exists sub02; CREATE database sub02 FROM sys PUBLICATION pub02; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub02 sys test02 rs02 2024-11-15 11:34:08 sub02 2024-11-15 11:34:08 0 +pub02 sys test02 rs02 2024-12-31 11:53:40 sub02 2024-12-31 11:53:40 0 drop snapshot if exists sp02; create snapshot sp02 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sp02 2024-11-15 03:34:08.377778 account acc01 +sp02 2024-12-31 03:53:41.016163 account acc01 drop database sub02; drop PUBLICATION pub02; show subscriptions; @@ -195,7 +195,7 @@ drop PUBLICATION if exists pub03; create PUBLICATION pub03 database test03 account acc01; show PUBLICATIONS; publication database tables sub_account subscribed_accounts create_time update_time comments -pub03 test03 * acc01 2024-11-15 11:34:08 null +pub03 test03 * acc01 2024-12-31 11:53:42 null show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status drop database if exists sub03; @@ -261,14 +261,14 @@ drop PUBLICATION if exists pub04; create PUBLICATION pub04 database test04 account sys comment 'pub to sys account'; show PUBLICATIONS; publication database tables sub_account subscribed_accounts create_time update_time comments -pub04 test04 * sys 2024-11-15 11:34:09 null pub to sys account +pub04 test04 * sys 2024-12-31 11:53:45 null pub to sys account drop database if exists sub05; create DATABASE sub05 from acc01 PUBLICATION pub04; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub04 acc01 test04 * pub to sys account 2024-11-15 11:34:09 sub05 2024-11-15 11:34:09 0 +pub04 acc01 test04 * pub to sys account 2024-12-31 11:53:45 sub05 2024-12-31 11:53:45 0 drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; drop database sub05; restore account sys database sub05 table t4 from snapshot sp05; internal error: can't restore to table for sub db @@ -290,7 +290,7 @@ count(*) 1 show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub04 acc01 test04 * pub to sys account 2024-12-17 16:00:47 sub05 2024-12-17 16:00:48 0 +pub04 acc01 test04 * pub to sys account 2024-12-31 11:53:45 sub05 2024-12-31 11:53:45 0 drop database if exists sub05; drop snapshot if exists sp05; drop PUBLICATION pub04; @@ -315,25 +315,25 @@ drop PUBLICATION if exists pub06; create PUBLICATION pub06 database test06 account acc02 comment 'acc01 pub to acc02'; show PUBLICATIONS; publication database tables sub_account subscribed_accounts create_time update_time comments -pub06 test06 * acc02 2024-11-15 11:34:09 null acc01 pub to acc02 +pub06 test06 * acc02 2024-12-31 11:53:46 null acc01 pub to acc02 drop database if exists sub06; CREATE database sub06 FROM acc01 PUBLICATION pub06; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub06 acc01 test06 * acc01 pub to acc02 2024-11-15 11:34:09 sub06 2024-11-15 11:34:09 0 +pub06 acc01 test06 * acc01 pub to acc02 2024-12-31 11:53:46 sub06 2024-12-31 11:53:46 0 restore account acc01 from snapshot sp06; show PUBLICATIONs; publication database tables sub_account subscribed_accounts create_time update_time comments show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub06 acc01 null null null null sub06 2024-11-15 11:34:09 2 +pub06 acc01 null null null null sub06 2024-12-31 11:53:46 2 use sub06; internal error: there is no publication pub06 drop PUBLICATION if exists pub06; create PUBLICATION pub06 database test06 table table01 account acc02 comment 'acc01 pub to acc02'; show PUBLICATIONS; publication database tables sub_account subscribed_accounts create_time update_time comments -pub06 test06 table01 acc02 acc02 2024-11-15 11:34:10 null acc01 pub to acc02 +pub06 test06 table01 acc02 acc02 2024-12-31 11:53:46 null acc01 pub to acc02 show databases; Database information_schema @@ -394,17 +394,17 @@ drop database if exists sub06; create database sub06 from sys PUBLICATION pub06; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub06 sys test07 * 2024-11-15 11:34:10 sub06 2024-11-15 11:34:10 0 +pub06 sys test07 * 2024-12-31 11:53:50 sub06 2024-12-31 11:53:50 0 drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; alter PUBLICATION pub06 account acc02 database test07; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub06 sys null null null null sub06 2024-11-15 11:34:10 1 +pub06 sys null null null null sub06 2024-12-31 11:53:50 1 restore account sys from snapshot sp06; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub06 sys null null null null sub06 2024-11-18 10:54:13 2 +pub06 sys null null null null sub06 2024-12-31 11:53:50 2 drop database sub06; drop PUBLICATION pub06; internal error: publication 'pub06' does not exist @@ -427,7 +427,7 @@ insert into table01 values (2, -1, null); drop PUBLICATION if exists pub10; create PUBLICATION pub10 database test01 table table01 account acc01; drop snapshot if exists sp10; -create snapshot sp10 for account sys; +create snapshot sp10 for account; drop table if exists table02; create table table02 (col1 int, col3 decimal); insert into table02 values(1, 2); @@ -442,11 +442,11 @@ show PUBLICATIONs; publication database tables sub_account subscribed_accounts create_time update_time comments show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub10 sys null null null null sub01 2024-11-18 10:54:13 2 +pub10 sys null null null null sub01 2024-12-31 11:53:56 2 drop database sub01; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub11 sys null null null null sub02 2024-11-15 11:34:10 2 +pub11 sys null null null null sub02 2024-12-31 11:53:56 2 drop database sub02; drop PUBLICATION pub10; internal error: publication 'pub10' does not exist diff --git a/test/distributed/cases/snapshot/restote_sub.sql b/test/distributed/cases/snapshot/restore_sub.sql similarity index 99% rename from test/distributed/cases/snapshot/restote_sub.sql rename to test/distributed/cases/snapshot/restore_sub.sql index 09fe170e8515e..ab1738ec43fcd 100644 --- a/test/distributed/cases/snapshot/restote_sub.sql +++ b/test/distributed/cases/snapshot/restore_sub.sql @@ -262,7 +262,7 @@ create DATABASE sub05 from acc01 PUBLICATION pub04; -- @ignore:5,7 show subscriptions; drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; drop database sub05; restore account sys database sub05 table t4 from snapshot sp05; @@ -386,7 +386,7 @@ show subscriptions; -- @session drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; alter PUBLICATION pub06 account acc02 database test07; @@ -431,7 +431,7 @@ drop PUBLICATION if exists pub10; create PUBLICATION pub10 database test01 table table01 account acc01; drop snapshot if exists sp10; -create snapshot sp10 for account sys; +create snapshot sp10 for account; drop table if exists table02; create table table02 (col1 int, col3 decimal); diff --git a/test/distributed/cases/snapshot/snapshot.result b/test/distributed/cases/snapshot/snapshot.result index 53420372b0825..d8611e9ba1bc7 100644 --- a/test/distributed/cases/snapshot/snapshot.result +++ b/test/distributed/cases/snapshot/snapshot.result @@ -11,21 +11,21 @@ sleep(1) create snapshot snapshot_04 for account default_1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot_04 2024-03-06 15:09:15 account default_1 -snapshot_03 2024-03-06 15:09:14 account default_1 -snapshot_01 2024-03-06 15:09:13 cluster -snapshot_02 2024-03-06 15:09:13 account default_1 +snapshot_04 2024-12-31 03:50:38.729708 account default_1 +snapshot_03 2024-12-31 03:50:37.616675 account default_1 +snapshot_02 2024-12-31 03:50:36.543137 account default_1 +snapshot_01 2024-12-31 03:50:36.145389 cluster show snapshots where SNAPSHOT_NAME = 'snapshot_01'; snapshot_name timestamp snapshot_level account_name database_name table_name -snapshot_01 2024-03-06 15:09:13 cluster +snapshot_01 2024-12-31 03:50:36.145389 cluster show snapshots where SNAPSHOT_LEVEL = 'cluster'; snapshot_name timestamp snapshot_level account_name database_name table_name -snapshot_01 2024-03-06 15:09:13 cluster +snapshot_01 2024-12-31 03:50:36.145389 cluster show snapshots where ACCOUNT_NAME = 'default_1'; snapshot_name timestamp snapshot_level account_name database_name table_name -snapshot_04 2024-03-06 15:09:15 account default_1 -snapshot_03 2024-03-06 15:09:14 account default_1 -snapshot_02 2024-03-06 15:09:13 account default_1 +snapshot_04 2024-12-31 03:50:38.729708 account default_1 +snapshot_03 2024-12-31 03:50:37.616675 account default_1 +snapshot_02 2024-12-31 03:50:36.543137 account default_1 DROP SNAPSHOT snapshot_01; DROP SNAPSHOT snapshot_02; DROP SNAPSHOT snapshot_03; @@ -41,22 +41,22 @@ sleep(1) create snapshot snapshot_07 for account default_1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot_07 2024-03-06 15:09:18 account default_1 -snapshot_06 2024-03-06 15:09:17 account default_1 -snapshot_05 2024-03-06 15:09:16 account default_1 +snapshot_07 2024-12-31 03:50:41.042356 account default_1 +snapshot_06 2024-12-31 03:50:40.013175 account default_1 +snapshot_05 2024-12-31 03:50:38.972736 account default_1 show snapshots where SNAPSHOT_NAME = 'snapshot_07'; snapshot_name timestamp snapshot_level account_name database_name table_name -snapshot_07 2024-03-06 15:09:18 account default_1 +snapshot_07 2024-12-31 03:50:41.042356 account default_1 show snapshots where SNAPSHOT_LEVEL = 'account'; snapshot_name timestamp snapshot_level account_name database_name table_name -snapshot_07 2024-03-06 15:09:18 account default_1 -snapshot_06 2024-03-06 15:09:17 account default_1 -snapshot_05 2024-03-06 15:09:16 account default_1 +snapshot_07 2024-12-31 03:50:41.042356 account default_1 +snapshot_06 2024-12-31 03:50:40.013175 account default_1 +snapshot_05 2024-12-31 03:50:38.972736 account default_1 show snapshots where ACCOUNT_NAME = 'default_1'; snapshot_name timestamp snapshot_level account_name database_name table_name -snapshot_07 2024-08-26 06:17:27.368063 account default_1 -snapshot_06 2024-08-26 06:17:26.352212 account default_1 -snapshot_05 2024-08-26 06:17:25.32872 account default_1 +snapshot_07 2024-12-31 03:50:41.042356 account default_1 +snapshot_06 2024-12-31 03:50:40.013175 account default_1 +snapshot_05 2024-12-31 03:50:38.972736 account default_1 drop account default_1; create snapshot snapshot_08 for account default_1; internal error: account default_1 does not exist @@ -71,8 +71,8 @@ create account default_2 ADMIN_NAME admin IDENTIFIED BY '111111'; create snapshot snapshot_10 for account default_2; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot_10 2024-03-06 15:09:20 account default_2 -snapshot_09 2024-03-06 15:09:18 account default_1 +snapshot_10 2024-12-31 03:50:44.821212 account default_2 +snapshot_09 2024-12-31 03:50:41.827006 account default_1 create snapshot snapshot_11 for account default_1; create snapshot snapshot_12 for account default_2; internal error: only sys tenant can create tenant level snapshot for other tenant @@ -80,7 +80,7 @@ create snapshot snapshot_13 for cluster; internal error: only sys tenant can create cluster level snapshot show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot_11 2024-03-06 15:09:20 account default_1 +snapshot_11 2024-12-31 03:50:44.884872 account default_1 create user efg identified by '111'; create snapshot snapshot_14 for account default_1; internal error: do not have privilege to execute the statement @@ -98,7 +98,7 @@ col1 col2 1 2 2 3 drop snapshot if exists `binary`; -create snapshot `binary` for account sys; +create snapshot `binary` for account; select count(*) from cluster01{snapshot = `binary`}; count(*) 2 @@ -116,7 +116,7 @@ col1 col2 1 2 2 3 drop snapshot if exists `_binary`; -create snapshot `_binary` for account sys; +create snapshot `_binary` for account; select count(*) from cluster01{snapshot = `_binary`}; count(*) 2 diff --git a/test/distributed/cases/snapshot/snapshot.sql b/test/distributed/cases/snapshot/snapshot.sql index d53acf5c35719..810c6c5ef47a1 100644 --- a/test/distributed/cases/snapshot/snapshot.sql +++ b/test/distributed/cases/snapshot/snapshot.sql @@ -70,7 +70,7 @@ insert into cluster01 values(1,2); insert into cluster01 values(2,3); select * from cluster01; drop snapshot if exists `binary`; -create snapshot `binary` for account sys; +create snapshot `binary` for account; select count(*) from cluster01{snapshot = `binary`}; restore account sys from snapshot `binary`; select count(*) from cluster01{snapshot = `binary`}; @@ -82,7 +82,7 @@ insert into cluster01 values(1,2); insert into cluster01 values(2,3); select * from cluster01; drop snapshot if exists `_binary`; -create snapshot `_binary` for account sys; +create snapshot `_binary` for account; select count(*) from cluster01{snapshot = `_binary`}; restore account sys from snapshot `_binary`; select count(*) from cluster01{snapshot = `_binary`}; diff --git a/test/distributed/cases/snapshot/snapshotRead.result b/test/distributed/cases/snapshot/snapshotRead.result index fa3e4c6747806..4ff40dbb72f42 100644 --- a/test/distributed/cases/snapshot/snapshotRead.result +++ b/test/distributed/cases/snapshot/snapshotRead.result @@ -32,7 +32,7 @@ show create table snapshot01; Table Create Table snapshot01 CREATE TABLE `snapshot01` (\n `col1` int NOT NULL,\n `col2` decimal(38,0) DEFAULT NULL,\n `col3` bigint DEFAULT NULL,\n `col4` double DEFAULT NULL,\n `col5` float DEFAULT NULL,\n PRIMARY KEY (`col1`)\n) drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; select * from snapshot01 {snapshot = 'sp01'}; col1 col2 col3 col4 col5 1 11 1234567890 123.45 678.9 @@ -50,7 +50,7 @@ count(*) 10 show snapshots where SNAPSHOT_NAME = 'sp01'; snapshot_name timestamp snapshot_level account_name database_name table_name -sp01 2024-10-28 10:09:01.873904373 account sys +sp01 2024-12-31 03:35:21.715416 account sys insert into snapshot01 values(11, 100.00, 8899001122, 101.24, 156.79); select count(*) from snapshot01; count(*) @@ -181,7 +181,7 @@ count(*) 1 select * from mo_catalog.mo_database{snapshot = 'sp01'} where datname = 'test'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -274717 test def create database test 0 1 2024-10-28 18:09:01 0 +283733 test def create database test 0 1 2024-12-31 11:35:21 0 select attname from mo_catalog.mo_columns{snapshot = 'sp01'} where att_database = 'test'; attname col1 @@ -205,8 +205,8 @@ insert into test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), select count(*) from test_snapshot_read; count(*) 100 -create snapshot snapshot_01 for account sys; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; +create snapshot snapshot_01 for account; internal error: snapshot snapshot_01 already exists drop snapshot snapshot_01; drop table test_snapshot_read; @@ -239,14 +239,14 @@ col1 col2 col3 1 1 1 2 2 2 drop snapshot if exists sp03; -create snapshot sp03 for account sys; +create snapshot sp03 for account; show snapshots where account_name = 'sys'; snapshot_name timestamp snapshot_level account_name database_name table_name -sp03 2024-10-28 10:09:02.060237872 account sys +sp03 2024-12-31 03:35:22.601626 account sys insert into snapshot03 values(3,'sfhuwe',1,1); insert into snapshot03 values(4,'37829901k3d',2,2); drop snapshot if exists sp04; -create snapshot sp04 for account sys; +create snapshot sp04 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME sp04 2024-10-28 10:09:02.08058989 account sys @@ -293,7 +293,7 @@ select count(*) from snapshot03 {snapshot = 'sp03'}; count(*) 2 drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME sp05 2024-10-28 10:09:02.122807362 account sys @@ -354,7 +354,7 @@ drop snapshot if exists snap01; create snapshot snap01 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snap01 2024-10-28 10:09:02.571420221 account acc01 +snap01 2024-12-31 03:35:23.69954 account acc01 use test01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME @@ -365,7 +365,7 @@ count(*) 1 select * from mo_catalog.mo_database{snapshot = 'snap01'} where datname = 'test01'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -274805 test01 def create database test01 2 2 2024-10-28 18:09:02 10 +283820 test01 def create database test01 2 2 2024-12-31 11:35:23 67 select attname from mo_catalog.mo_columns{snapshot = 'snap01'} where att_database = 'test01'; attname col1 @@ -379,7 +379,7 @@ count(*) 1 select * from mo_catalog.mo_database{snapshot = 'snap01'} where datname = 'test01'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -274805 test01 def create database test01 2 2 2024-10-28 18:09:02 10 +283820 test01 def create database test01 2 2 2024-12-31 11:35:23 67 select attname from mo_catalog.mo_columns{snapshot = 'snap01'} where att_database = 'test01'; attname col1 @@ -406,10 +406,10 @@ drop snapshot if exists snap02; create snapshot snap02 for account acc02; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snap02 2024-10-28 10:09:03.141989874 account acc02 +snap02 2024-12-31 03:35:24.853436 account acc02 show snapshots where account_name = 'acc02'; snapshot_name timestamp snapshot_level account_name database_name table_name -snap02 2024-10-28 10:09:03.141989874 account acc02 +snap02 2024-12-31 03:35:24.853436 account acc02 select count(*) from acc02_test01 {snapshot = 'snap02'}; count(*) 3 @@ -453,7 +453,7 @@ count(*) 1 select * from mo_catalog.mo_database{snapshot = 'snap02'} where datname = 'test02'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -274885 test02 def create database test02 2 2 2024-10-28 18:09:03 11 +283899 test02 def create database test02 2 2 2024-12-31 11:35:24 68 select attname from mo_catalog.mo_columns{snapshot = 'snap02'} where att_database = 'test02'; attname col1 @@ -496,7 +496,7 @@ drop snapshot if exists testsnap_03; select count(*) from testsnap_03; count(*) 10 -create snapshot sp04 for account sys; +create snapshot sp04 for account; select count(*) from testsnap_03 {snapshot = 'sp04'}; count(*) 10 @@ -509,7 +509,7 @@ select count(*) from testsnap_03 {snapshot = 'sp04'}; count(*) 10 drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; select * from testsnap_03; employeenumber lastname firstname extension email officecode reportsto jobtitle 1002 Murphy Diane x5800 dmurphy@classicmodelcars.com 1 null President @@ -573,7 +573,7 @@ count(*) 2 select * from mo_catalog.mo_database{snapshot = 'sp04'} where datname = 'test03'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -274889 test03 def create database test03 0 1 2024-10-28 18:09:03 0 +283903 test03 def create database test03 0 1 2024-12-31 11:35:25 0 select attname from mo_catalog.mo_columns{snapshot = 'sp04'} where att_database = 'test03'; attname employeeNumber @@ -604,13 +604,13 @@ col1 col2 account_id 1 2 0 2 3 0 drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; select count(*) from mo_catalog.mo_tables{snapshot = sp06} where reldatabase = 'mo_catalog'; count(*) 48 select * from mo_catalog.mo_database{snapshot = sp06} where datname = 'mo_catalog'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -1 mo_catalog mo_catalog 0 0 2024-10-28 18:07:36 0 +1 mo_catalog mo_catalog 0 0 2024-12-30 10:46:53 0 drop table cluster01; drop snapshot sp06; drop database if exists test03; @@ -638,7 +638,7 @@ show tables; Tables_in_sub_database01 pub01 drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; select * from pub01 {snapshot = 'sp06'}; col1 col2 col3 col4 col5 1 11 1234567890 123.45 678.9 @@ -653,7 +653,7 @@ count(*) 1 select * from mo_catalog.mo_database{snapshot = 'sp06'} where datname = 'test03'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -274894 test03 def create database test03 0 1 2024-10-28 18:09:03 0 +283908 test03 def create database test03 0 1 2024-12-31 11:35:26 0 select attname from mo_catalog.mo_columns{snapshot = 'sp06'} where att_database = 'test03'; attname col1 @@ -680,10 +680,10 @@ select count(*) from pt_table; count(*) 11 drop snapshot if exists sp07; -create snapshot sp07 for account sys; +create snapshot sp07 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sp07 2024-10-28 10:09:04.235206461 account sys +sp07 2024-12-31 03:35:31.619311 account sys select count(*) from pt_table{snapshot = 'sp07'}; count(*) 11 @@ -695,7 +695,7 @@ count(*) 11 select * from mo_catalog.mo_database{snapshot = 'sp07'} where datname = 'test03'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -274894 test03 def create database test03 0 1 2024-10-28 18:09:03 0 +283908 test03 def create database test03 0 1 2024-12-31 11:35:26 0 select attname from mo_catalog.mo_columns{snapshot = 'sp07'} where att_database = 'test03'; attname col1 @@ -979,7 +979,7 @@ create publication publication01 database test04 account test_tenant_1 comment ' create database sub_database02 from sys publication publication01; use sub_database02; drop snapshot if exists sp08; -create snapshot sp08 for account sys; +create snapshot sp08 for account; use test04; select count(*) from normal_table01 {snapshot = 'sp08'}; count(*) @@ -1014,7 +1014,7 @@ count(*) drop account test_tenant_1; select * from mo_catalog.mo_database{snapshot = 'sp08'} where datname = 'test04'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -274986 test04 def create database test04 0 1 2024-10-28 18:09:04 0 +283999 test04 def create database test04 0 1 2024-12-31 11:35:31 0 select attname from mo_catalog.mo_columns{snapshot = 'sp08'} where att_database = 'test04'; attname col1 @@ -1033,7 +1033,7 @@ count(*) 3 select * from mo_catalog.mo_database{snapshot = 'sp08'} where datname = 'test04'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -274986 test04 def create database test04 0 1 2024-10-28 18:09:04 0 +283999 test04 def create database test04 0 1 2024-12-31 11:35:31 0 select attname from mo_catalog.mo_columns{snapshot = 'sp08'} where att_database = 'test04'; attname col1 @@ -1070,13 +1070,13 @@ select count(*) from t1; count(*) 7 drop snapshot if exists `binary`; -create snapshot `binary` for account sys; +create snapshot `binary` for account; select count(*) from mo_catalog.mo_tables{snapshot = `binary`} where reldatabase = 'test05'; count(*) 1 select * from mo_catalog.mo_database{snapshot = `binary`} where datname = 'test05'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -275069 test05 def create database test05 0 1 2024-10-28 18:09:05 0 +284081 test05 def create database test05 0 1 2024-12-31 11:35:40 0 select attname from mo_catalog.mo_columns{snapshot = `binary`} where att_database = 'test05'; attname a @@ -1093,13 +1093,13 @@ primary key(dvalue) insert into t1 values('2022-01-01', 1); insert into t1 values('2022-01-02', 2); drop snapshot if exists consistent; -create snapshot consistent for account sys; +create snapshot consistent for account; select count(*) from mo_catalog.mo_tables{snapshot = consistent} where reldatabase = 'test05'; count(*) 1 select * from mo_catalog.mo_database{snapshot = consistent} where datname = 'test05'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -275069 test05 def create database test05 0 1 2024-10-28 18:09:05 0 +284081 test05 def create database test05 0 1 2024-12-31 11:35:40 0 select attname from mo_catalog.mo_columns{snapshot = consistent} where att_database = 'test05'; attname dvalue @@ -1115,13 +1115,13 @@ drop table if exists tran01; start transaction; create table tran01(col1 enum('red','blue','green')); insert into tran01 values('red'),('blue'),('green'); -create snapshot sp09 for account sys; +create snapshot sp09 for account; internal error: unclassified statement appears in uncommitted transaction show snapshots; internal error: unclassified statement appears in uncommitted transaction commit; drop snapshot if exists sp09; -create snapshot sp09 for account sys; +create snapshot sp09 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME sp09 2024-10-28 10:09:05.169634826 account sys @@ -1134,7 +1134,7 @@ count(*) 1 select * from mo_catalog.mo_database{snapshot = 'sp09'} where datname = 'test06'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -275072 test06 def create database test06 0 1 2024-10-28 18:09:05 0 +284085 test06 def create database test06 0 1 2024-12-31 11:35:41 0 select attname from mo_catalog.mo_columns{snapshot = 'sp09'} where att_database = 'test06'; attname col1 @@ -1189,7 +1189,7 @@ deptno dname loc drop table t3; drop table t2; drop snapshot if exists sp10; -create snapshot sp10 for account sys; +create snapshot sp10 for account; select count(*) from t3{snapshot = 'sp10'}; SQL parser error: table "t3" does not exist select count(*) from t2{snapshot = 'sp10'}; diff --git a/test/distributed/cases/snapshot/snapshotRead.sql b/test/distributed/cases/snapshot/snapshotRead.sql index c7515a73086d2..c87fd76f74ae9 100644 --- a/test/distributed/cases/snapshot/snapshotRead.sql +++ b/test/distributed/cases/snapshot/snapshotRead.sql @@ -19,7 +19,7 @@ select count(*) from snapshot01; select * from snapshot01; show create table snapshot01; drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; select * from snapshot01 {snapshot = 'sp01'}; select count(*) from snapshot01 {snapshot = 'sp01'}; @@ -74,8 +74,8 @@ drop table if exists snapshot02; create table test_snapshot_read (a int); insert into test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40),(41), (42), (43), (44), (45), (46), (47), (48), (49), (50),(51), (52), (53), (54), (55), (56), (57), (58), (59), (60),(61), (62), (63), (64), (65), (66), (67), (68), (69), (70),(71), (72), (73), (74), (75), (76), (77), (78), (79), (80), (81), (82), (83), (84), (85), (86), (87), (88), (89), (90),(91), (92), (93), (94), (95), (96), (97), (98), (99), (100); select count(*) from test_snapshot_read; -create snapshot snapshot_01 for account sys; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; +create snapshot snapshot_01 for account; drop snapshot snapshot_01; drop table test_snapshot_read; @@ -100,13 +100,13 @@ select * from snapshot03; select * from snapshot04; drop snapshot if exists sp03; -create snapshot sp03 for account sys; +create snapshot sp03 for account; -- @ignore:1 show snapshots where account_name = 'sys'; insert into snapshot03 values(3,'sfhuwe',1,1); insert into snapshot03 values(4,'37829901k3d',2,2); drop snapshot if exists sp04; -create snapshot sp04 for account sys; +create snapshot sp04 for account; -- @ignore:1 show snapshots; select * from snapshot03; @@ -122,7 +122,7 @@ insert into snapshot03 values(5,'sfhuwe',1); select * from snapshot03; select count(*) from snapshot03 {snapshot = 'sp03'}; drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; -- @ignore:1 show snapshots; select count(*) from snapshot03 {snapshot = 'sp05'}; @@ -247,7 +247,7 @@ insert into testsnap_03(employeeNumber,lastName,firstName,extension,email,office drop snapshot if exists testsnap_03; select count(*) from testsnap_03; -create snapshot sp04 for account sys; +create snapshot sp04 for account; select count(*) from testsnap_03 {snapshot = 'sp04'}; insert into testsnap_03 values (1286,'Tseng','Foon Yue','x2248','ftseng@classicmodelcars.com','3',1143,'Sales Rep'), (1323,'Vanauf','George','x4102','gvanauf@classicmodelcars.com','3',1143,'Sales Rep'), @@ -256,7 +256,7 @@ insert into testsnap_03 values (1286,'Tseng','Foon Yue','x2248','ftseng@classicm (1401,'Castillo','Pamela','x2759','pcastillo@classicmodelcars.com','4',1102,'Sales Rep'); select count(*) from testsnap_03 {snapshot = 'sp04'}; drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; select * from testsnap_03; select * from testsnap_03 {snapshot = 'sp04'} where jobTitle = 'President'; select employeeNumber,lastName,firstName,extension from testsnap_03 {snapshot = 'sp04'} where extension = 'x4611'; @@ -289,7 +289,7 @@ insert into cluster01 values(1,2,0); insert into cluster01 values(2,3,0); select * from cluster01; drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; select count(*) from mo_catalog.mo_tables{snapshot = sp06} where reldatabase = 'mo_catalog'; -- @ignore:0,6,7 select * from mo_catalog.mo_database{snapshot = sp06} where datname = 'mo_catalog'; @@ -315,7 +315,7 @@ use sub_database01; show tables; -- @session drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; select * from pub01 {snapshot = 'sp06'}; select count(*) from pub01 {snapshot = 'sp06'}; select count(*) from mo_catalog.mo_tables{snapshot = 'sp06'} where reldatabase = 'test03'; @@ -340,7 +340,7 @@ create table pt_table(col1 tinyint,col2 smallint,col3 int,clo4 bigint,col5 tinyi load data infile '$resources/external_table_file/pt_table_data.csv' into table pt_table fields terminated by ','; select count(*) from pt_table; drop snapshot if exists sp07; -create snapshot sp07 for account sys; +create snapshot sp07 for account; -- @ignore:1 show snapshots; select count(*) from pt_table{snapshot = 'sp07'}; @@ -385,7 +385,7 @@ create database sub_database02 from sys publication publication01; use sub_database02; -- @session drop snapshot if exists sp08; -create snapshot sp08 for account sys; +create snapshot sp08 for account; use test04; select count(*) from normal_table01 {snapshot = 'sp08'}; select count(*) from t3 {snapshot = 'sp08'}; @@ -434,7 +434,7 @@ insert into t1 values('abcd_f'); insert into t1 values('abcde_'); select count(*) from t1; drop snapshot if exists `binary`; -create snapshot `binary` for account sys; +create snapshot `binary` for account; select count(*) from mo_catalog.mo_tables{snapshot = `binary`} where reldatabase = 'test05'; -- @ignore:0,6,7 select * from mo_catalog.mo_database{snapshot = `binary`} where datname = 'test05'; @@ -451,7 +451,7 @@ create table t1 ( insert into t1 values('2022-01-01', 1); insert into t1 values('2022-01-02', 2); drop snapshot if exists consistent; -create snapshot consistent for account sys; +create snapshot consistent for account; -- @ignore:0,6,7 select count(*) from mo_catalog.mo_tables{snapshot = consistent} where reldatabase = 'test05'; -- @ignore:0,6,7 @@ -469,12 +469,12 @@ drop table if exists tran01; start transaction; create table tran01(col1 enum('red','blue','green')); insert into tran01 values('red'),('blue'),('green'); -create snapshot sp09 for account sys; +create snapshot sp09 for account; -- @ignore:1 show snapshots; commit; drop snapshot if exists sp09; -create snapshot sp09 for account sys; +create snapshot sp09 for account; -- @ignore:1 show snapshots; select count(*) from tran01{snapshot = 'sp09'}; @@ -521,7 +521,7 @@ drop table t3; drop table t2; drop snapshot if exists sp10; -create snapshot sp10 for account sys; +create snapshot sp10 for account; select count(*) from t3{snapshot = 'sp10'}; select count(*) from t2{snapshot = 'sp10'}; diff --git a/test/distributed/cases/snapshot/snapshot_database_level.result b/test/distributed/cases/snapshot/snapshot_database_level.result index cd9aec102e46a..653722565eaf7 100644 --- a/test/distributed/cases/snapshot/snapshot_database_level.result +++ b/test/distributed/cases/snapshot/snapshot_database_level.result @@ -8,7 +8,7 @@ create database if not exists db1; create snapshot sn2 for database db1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sn2 2024-12-20 02:58:51.442498 database sys db1 +sn2 2024-12-31 06:16:57.71952 database sys db1 drop database if exists db1; drop snapshot if exists sn2; drop snapshot if exists sn1; @@ -26,7 +26,7 @@ create database if not exists db1; create snapshot sn2 for database db1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sn2 2024-12-20 02:58:51.951431 database acc01 db1 +sn2 2024-12-31 06:16:58.293388 database acc01 db1 drop database if exists db1; drop snapshot if exists sn2; drop snapshot if exists sn1; @@ -43,7 +43,7 @@ insert into db1.tbl1 values (1), (2), (3); create snapshot sn2 for table db1 tbl1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sn2 2024-12-20 02:58:52.013035 table sys db1 tbl1 +sn2 2024-12-31 06:16:58.616386 table sys db1 tbl1 drop database if exists db1; drop snapshot if exists sn2; drop snapshot if exists sn1; @@ -63,15 +63,15 @@ insert into db1.tbl1 values (1), (2), (3); create snapshot sn2 for table db1 tbl1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sn2 2024-12-20 02:58:52.689975 table acc01 db1 tbl1 +sn2 2024-12-31 06:16:59.522949 table acc01 db1 tbl1 drop database if exists db1; drop snapshot if exists sn2; drop snapshot if exists sn1; drop account if exists acc1; -create snapshot sn1 for account sys; -create snapshot sn1 for account sys; +create snapshot sn1 for account; +create snapshot sn1 for account; internal error: snapshot sn1 already exists -create snapshot if not exists sn1 for account sys; +create snapshot if not exists sn1 for account; drop snapshot if exists sn1; create snapshot sn1 for database mo_catalog; internal error: can not create snapshot for current database mo_catalog @@ -82,5 +82,5 @@ drop snapshot if exists sn1; create snapshot sn1 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sn1 2024-12-25 07:36:53.495417 account sys +sn1 2024-12-31 06:16:59.861163 account sys drop snapshot if exists sn1; diff --git a/test/distributed/cases/snapshot/snapshot_database_level.sql b/test/distributed/cases/snapshot/snapshot_database_level.sql index eb782825f94cc..f32331e6bef5a 100644 --- a/test/distributed/cases/snapshot/snapshot_database_level.sql +++ b/test/distributed/cases/snapshot/snapshot_database_level.sql @@ -80,9 +80,9 @@ drop snapshot if exists sn1; drop account if exists acc1; -create snapshot sn1 for account sys; -create snapshot sn1 for account sys; -create snapshot if not exists sn1 for account sys; +create snapshot sn1 for account; +create snapshot sn1 for account; +create snapshot if not exists sn1 for account; drop snapshot if exists sn1; diff --git a/test/distributed/cases/snapshot/snapshot_read.result b/test/distributed/cases/snapshot/snapshot_read.result index 7eb721d4a4e68..d9f8b3f7505ea 100644 --- a/test/distributed/cases/snapshot/snapshot_read.result +++ b/test/distributed/cases/snapshot/snapshot_read.result @@ -8,7 +8,7 @@ count(*) select sleep(1); sleep(1) 0 -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; delete from test_snapshot_read where a <= 50; select count(*) from test_snapshot_read; count(*) @@ -19,7 +19,7 @@ count(*) select sleep(1); sleep(1) 0 -create snapshot snapshot_02 for account sys; +create snapshot snapshot_02 for account; INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40); select count(*) from test_snapshot_read; count(*) @@ -38,7 +38,7 @@ count(*) select sleep(1); sleep(1) 0 -create snapshot snapshot_03 for account sys; +create snapshot snapshot_03 for account; drop table if exists test_snapshot_read2; select count(*) from mo_catalog.mo_tables where reldatabase = 'snapshot_read'; count(*) @@ -58,7 +58,7 @@ INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), select count(*) from test_snapshot_read; count(*) 100 -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; delete from test_snapshot_read where a <= 50; select count(*) from test_snapshot_read; count(*) @@ -66,7 +66,7 @@ count(*) select count(*) from test_snapshot_read {snapshot = 'snapshot_01'}; count(*) 100 -create snapshot snapshot_02 for account sys; +create snapshot snapshot_02 for account; INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40); select count(*) from test_snapshot_read; count(*) @@ -82,7 +82,7 @@ test_snapshot_read2 select count(*) from mo_catalog.mo_tables where reldatabase = 'snapshot_read'; count(*) 2 -create snapshot snapshot_03 for account sys; +create snapshot snapshot_03 for account; drop table if exists test_snapshot_read2; select count(*) from mo_catalog.mo_tables where reldatabase = 'snapshot_read'; count(*) @@ -163,7 +163,7 @@ id username email select id, username, email from users where email = 'alice.jones@gmail.com'; id username email 3 alice_jones alice.jones@gmail.com -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; DELETE FROM users where email = 'john@example.com'; UPDATE users SET password = 'newsecurepassword123' WHERE email = 'alice.jones@gmail.com'; select id, username, email from users where email = 'john@example.com'; diff --git a/test/distributed/cases/snapshot/snapshot_read.sql b/test/distributed/cases/snapshot/snapshot_read.sql index cf52fc1adac26..431f344c12e1e 100644 --- a/test/distributed/cases/snapshot/snapshot_read.sql +++ b/test/distributed/cases/snapshot/snapshot_read.sql @@ -4,12 +4,12 @@ create table test_snapshot_read (a int); INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40),(41), (42), (43), (44), (45), (46), (47), (48), (49), (50),(51), (52), (53), (54), (55), (56), (57), (58), (59), (60),(61), (62), (63), (64), (65), (66), (67), (68), (69), (70),(71), (72), (73), (74), (75), (76), (77), (78), (79), (80), (81), (82), (83), (84), (85), (86), (87), (88), (89), (90),(91), (92), (93), (94), (95), (96), (97), (98), (99), (100); select count(*) from test_snapshot_read; select sleep(1); -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; delete from test_snapshot_read where a <= 50; select count(*) from test_snapshot_read; select count(*) from test_snapshot_read {snapshot = 'snapshot_01'}; select sleep(1); -create snapshot snapshot_02 for account sys; +create snapshot snapshot_02 for account; INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40); select count(*) from test_snapshot_read; select count(*) from test_snapshot_read{snapshot = 'snapshot_02'}; @@ -18,7 +18,7 @@ INSERT INTO test_snapshot_read2 select * from test_snapshot_read{snapshot = 'sna select count(*) test_snapshot_read2; select count(*) from mo_catalog.mo_tables where reldatabase = 'snapshot_read'; select sleep(1); -create snapshot snapshot_03 for account sys; +create snapshot snapshot_03 for account; drop table if exists test_snapshot_read2; select count(*) from mo_catalog.mo_tables where reldatabase = 'snapshot_read'; select count(*) from mo_catalog.mo_tables{snapshot = 'snapshot_03'} where reldatabase = 'snapshot_read'; @@ -33,11 +33,11 @@ use snapshot_read; create table test_snapshot_read (a int); INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40),(41), (42), (43), (44), (45), (46), (47), (48), (49), (50),(51), (52), (53), (54), (55), (56), (57), (58), (59), (60),(61), (62), (63), (64), (65), (66), (67), (68), (69), (70),(71), (72), (73), (74), (75), (76), (77), (78), (79), (80), (81), (82), (83), (84), (85), (86), (87), (88), (89), (90),(91), (92), (93), (94), (95), (96), (97), (98), (99), (100); select count(*) from test_snapshot_read; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; delete from test_snapshot_read where a <= 50; select count(*) from test_snapshot_read; select count(*) from test_snapshot_read {snapshot = 'snapshot_01'}; -create snapshot snapshot_02 for account sys; +create snapshot snapshot_02 for account; INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40); select count(*) from test_snapshot_read; select count(*) from test_snapshot_read{snapshot = 'snapshot_02'}; @@ -45,7 +45,7 @@ create table test_snapshot_read2(b int); INSERT INTO test_snapshot_read2 select * from test_snapshot_read{snapshot = 'snapshot_01'} where a <= 30; select count(*) test_snapshot_read2; select count(*) from mo_catalog.mo_tables where reldatabase = 'snapshot_read'; -create snapshot snapshot_03 for account sys; +create snapshot snapshot_03 for account; drop table if exists test_snapshot_read2; select count(*) from mo_catalog.mo_tables where reldatabase = 'snapshot_read'; select count(*) from mo_catalog.mo_tables{snapshot = 'snapshot_03'} where reldatabase = 'snapshot_read'; @@ -105,7 +105,7 @@ INSERT INTO users (username, email, password) VALUES ('emily_adams', 'emily.adam select id, username, email from users where email = 'john@example.com'; select id, username, email from users where email = 'alice.jones@gmail.com'; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; DELETE FROM users where email = 'john@example.com'; UPDATE users SET password = 'newsecurepassword123' WHERE email = 'alice.jones@gmail.com'; diff --git a/test/distributed/cases/snapshot/snapshot_read_1.result b/test/distributed/cases/snapshot/snapshot_read_1.result index bd5b3686caf8f..e79db3015a10a 100644 --- a/test/distributed/cases/snapshot/snapshot_read_1.result +++ b/test/distributed/cases/snapshot/snapshot_read_1.result @@ -1,7 +1,7 @@ -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; create database if not exists snapshot_read; use snapshot_read; -create snapshot snapshot_02 for account sys; +create snapshot snapshot_02 for account; create table test_snapshot_read (a int); INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40),(41), (42), (43), (44), (45), (46), (47), (48), (49), (50),(51), (52), (53), (54), (55), (56), (57), (58), (59), (60),(61), (62), (63), (64), (65), (66), (67), (68), (69), (70),(71), (72), (73), (74), (75), (76), (77), (78), (79), (80), (81), (82), (83), (84), (85), (86), (87), (88), (89), (90),(91), (92), (93), (94), (95), (96), (97), (98), (99), (100); select count(*) from snapshot_read.test_snapshot_read; @@ -40,7 +40,7 @@ INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), select count(*) from test_snapshot_read; count(*) 100 -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; drop table test_snapshot_read; select count(*) from snapshot_read.test_snapshot_read{snapshot = 'snapshot_01'}; count(*) @@ -54,7 +54,7 @@ INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), select count(*) from test_snapshot_read; count(*) 100 -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; drop database snapshot_read; select count(*) from snapshot_read.test_snapshot_read{snapshot = 'snapshot_01'}; count(*) diff --git a/test/distributed/cases/snapshot/snapshot_read_1.sql b/test/distributed/cases/snapshot/snapshot_read_1.sql index a003d04064d1e..438587b60ab4e 100644 --- a/test/distributed/cases/snapshot/snapshot_read_1.sql +++ b/test/distributed/cases/snapshot/snapshot_read_1.sql @@ -1,7 +1,7 @@ -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; create database if not exists snapshot_read; use snapshot_read; -create snapshot snapshot_02 for account sys; +create snapshot snapshot_02 for account; create table test_snapshot_read (a int); INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40),(41), (42), (43), (44), (45), (46), (47), (48), (49), (50),(51), (52), (53), (54), (55), (56), (57), (58), (59), (60),(61), (62), (63), (64), (65), (66), (67), (68), (69), (70),(71), (72), (73), (74), (75), (76), (77), (78), (79), (80), (81), (82), (83), (84), (85), (86), (87), (88), (89), (90),(91), (92), (93), (94), (95), (96), (97), (98), (99), (100); select count(*) from snapshot_read.test_snapshot_read; @@ -36,7 +36,7 @@ use snapshot_read; create table test_snapshot_read (a int); INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40),(41), (42), (43), (44), (45), (46), (47), (48), (49), (50),(51), (52), (53), (54), (55), (56), (57), (58), (59), (60),(61), (62), (63), (64), (65), (66), (67), (68), (69), (70),(71), (72), (73), (74), (75), (76), (77), (78), (79), (80), (81), (82), (83), (84), (85), (86), (87), (88), (89), (90),(91), (92), (93), (94), (95), (96), (97), (98), (99), (100); select count(*) from test_snapshot_read; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; drop table test_snapshot_read; select count(*) from snapshot_read.test_snapshot_read{snapshot = 'snapshot_01'}; drop database if exists snapshot_read; @@ -48,7 +48,7 @@ use snapshot_read; create table test_snapshot_read (a int); INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40),(41), (42), (43), (44), (45), (46), (47), (48), (49), (50),(51), (52), (53), (54), (55), (56), (57), (58), (59), (60),(61), (62), (63), (64), (65), (66), (67), (68), (69), (70),(71), (72), (73), (74), (75), (76), (77), (78), (79), (80), (81), (82), (83), (84), (85), (86), (87), (88), (89), (90),(91), (92), (93), (94), (95), (96), (97), (98), (99), (100); select count(*) from test_snapshot_read; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; drop database snapshot_read; select count(*) from snapshot_read.test_snapshot_read{snapshot = 'snapshot_01'}; drop database if exists snapshot_read; diff --git a/test/distributed/cases/snapshot/snapshot_read_3.result b/test/distributed/cases/snapshot/snapshot_read_3.result index dc335103e5a15..6c164361d72c8 100644 --- a/test/distributed/cases/snapshot/snapshot_read_3.result +++ b/test/distributed/cases/snapshot/snapshot_read_3.result @@ -6,7 +6,7 @@ select count(*) from test_snapshot_read; count(*) 100 create view test_snapshot_read_view as select * from test_snapshot_read; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; show tables like'test%'; Tables_in_snapshot_read test_snapshot_read @@ -48,7 +48,7 @@ select count(*) from test_snapshot_read; count(*) 100 create view test_snapshot_read_view as select * from test_snapshot_read; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; show tables like'test%'; Tables_in_snapshot_read test_snapshot_read diff --git a/test/distributed/cases/snapshot/snapshot_read_3.sql b/test/distributed/cases/snapshot/snapshot_read_3.sql index 9e86c596dd8df..e8b3b5405d6c2 100644 --- a/test/distributed/cases/snapshot/snapshot_read_3.sql +++ b/test/distributed/cases/snapshot/snapshot_read_3.sql @@ -4,7 +4,7 @@ create table test_snapshot_read (a int); INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40),(41), (42), (43), (44), (45), (46), (47), (48), (49), (50),(51), (52), (53), (54), (55), (56), (57), (58), (59), (60),(61), (62), (63), (64), (65), (66), (67), (68), (69), (70),(71), (72), (73), (74), (75), (76), (77), (78), (79), (80), (81), (82), (83), (84), (85), (86), (87), (88), (89), (90),(91), (92), (93), (94), (95), (96), (97), (98), (99), (100); select count(*) from test_snapshot_read; create view test_snapshot_read_view as select * from test_snapshot_read; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; show tables like'test%'; show create table test_snapshot_read; show create table test_snapshot_read_view; @@ -27,7 +27,7 @@ create table test_snapshot_read (a int); INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40),(41), (42), (43), (44), (45), (46), (47), (48), (49), (50),(51), (52), (53), (54), (55), (56), (57), (58), (59), (60),(61), (62), (63), (64), (65), (66), (67), (68), (69), (70),(71), (72), (73), (74), (75), (76), (77), (78), (79), (80), (81), (82), (83), (84), (85), (86), (87), (88), (89), (90),(91), (92), (93), (94), (95), (96), (97), (98), (99), (100); select count(*) from test_snapshot_read; create view test_snapshot_read_view as select * from test_snapshot_read; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; show tables like'test%'; show create table test_snapshot_read; show create table test_snapshot_read_view; diff --git a/test/distributed/cases/snapshot/snapshot_read_incremental_data.result b/test/distributed/cases/snapshot/snapshot_read_incremental_data.result index d75452e8122d0..0acd97971ca50 100644 --- a/test/distributed/cases/snapshot/snapshot_read_incremental_data.result +++ b/test/distributed/cases/snapshot/snapshot_read_incremental_data.result @@ -34,7 +34,7 @@ insert into lineitem values select count(*) from lineitem; count(*) 10 -create snapshot snapshot_lineitem for account sys; +create snapshot snapshot_lineitem for account; create snapshot snapshot_lineitem_incremental for cluster; insert into lineitem values (11,3,30,5,4,2,1860.06,0.01,0.06,"A","F","1993-12-04","1994-01-07","1994-01-01","NONE","TRUCK","y. fluffily pending d"), @@ -85,7 +85,7 @@ select count(*) from rs01; count(*) 6 drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; select count(*) from rs01 {snapshot = 'sp01'}; count(*) 6 diff --git a/test/distributed/cases/snapshot/snapshot_read_incremental_data.sql b/test/distributed/cases/snapshot/snapshot_read_incremental_data.sql index ad2093cf5df73..a6c7b2d9eeb74 100644 --- a/test/distributed/cases/snapshot/snapshot_read_incremental_data.sql +++ b/test/distributed/cases/snapshot/snapshot_read_incremental_data.sql @@ -32,7 +32,7 @@ insert into lineitem values (9,3,20,10,2,49,45080.98,0.10,0.00,"R","F","1993-11-09","1993-12-20","1993-11-24","TAKE BACK RETURN","RAIL"," unusual accounts. eve"), (10,3,129,8,3,27,27786.24,0.06,0.07,"A","F","1994-01-16","1993-11-22","1994-01-23","DELIVER IN PERSON","SHIP","nal foxes wake. "); select count(*) from lineitem; -create snapshot snapshot_lineitem for account sys; +create snapshot snapshot_lineitem for account; create snapshot snapshot_lineitem_incremental for cluster; insert into lineitem values (11,3,30,5,4,2,1860.06,0.01,0.06,"A","F","1993-12-04","1994-01-07","1994-01-01","NONE","TRUCK","y. fluffily pending d"), @@ -75,7 +75,7 @@ insert into rs01 values (2, -1, null); select count(*) from rs01; drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; select count(*) from rs01 {snapshot = 'sp01'}; insert into rs01 values (2, -1, null); insert into rs01 values (1, -321.321, null); diff --git a/test/distributed/cases/snapshot/snapshot_restore_account_level.result b/test/distributed/cases/snapshot/snapshot_restore_account_level.result index eaa47690fa8fc..b040cc4e8f9fa 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_account_level.result +++ b/test/distributed/cases/snapshot/snapshot_restore_account_level.result @@ -120,7 +120,7 @@ INSERT INTO factories (factory_name, address) VALUES ('Factory WWWW', '2031 Birc select count(*) from test_snapshot_restore.factories; count(*) 100 -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; drop database test_snapshot_restore; drop database snapshot_read; restore account sys from snapshot snapshot_01; @@ -622,7 +622,7 @@ drop snapshot snapshot_01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME drop snapshot if exists snapshot_01; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; restore account sys database mo_catalog from snapshot snapshot_01; internal error: can't restore db: mo_catalog drop snapshot if exists snapshot_01; diff --git a/test/distributed/cases/snapshot/snapshot_restore_account_level.sql b/test/distributed/cases/snapshot/snapshot_restore_account_level.sql index 22836ae08053d..367d075a984f2 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_account_level.sql +++ b/test/distributed/cases/snapshot/snapshot_restore_account_level.sql @@ -122,7 +122,7 @@ INSERT INTO factories (factory_name, address) VALUES ('Factory WWWW', '2031 Birc select count(*) from test_snapshot_restore.factories; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; drop database test_snapshot_restore; @@ -616,7 +616,7 @@ drop snapshot snapshot_01; show snapshots; drop snapshot if exists snapshot_01; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; restore account sys database mo_catalog from snapshot snapshot_01; drop snapshot if exists snapshot_01; diff --git a/test/distributed/cases/snapshot/snapshot_restote_cluster_table.result b/test/distributed/cases/snapshot/snapshot_restore_cluster_table.result similarity index 96% rename from test/distributed/cases/snapshot/snapshot_restote_cluster_table.result rename to test/distributed/cases/snapshot/snapshot_restore_cluster_table.result index b3f3314eb28f2..85afcef2ab6d8 100644 --- a/test/distributed/cases/snapshot/snapshot_restote_cluster_table.result +++ b/test/distributed/cases/snapshot/snapshot_restore_cluster_table.result @@ -14,7 +14,7 @@ a b account_id select * from mo_catalog.cluster_table_1; a b create snapshot snapshot1 for account acc01; -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; drop table if exists cluster_table_1; select * from mo_catalog.cluster_table_1; SQL parser error: table "cluster_table_1" does not exist diff --git a/test/distributed/cases/snapshot/snapshot_restote_cluster_table.sql b/test/distributed/cases/snapshot/snapshot_restore_cluster_table.sql similarity index 96% rename from test/distributed/cases/snapshot/snapshot_restote_cluster_table.sql rename to test/distributed/cases/snapshot/snapshot_restore_cluster_table.sql index 29665a129c3b8..06bed3916a009 100644 --- a/test/distributed/cases/snapshot/snapshot_restote_cluster_table.sql +++ b/test/distributed/cases/snapshot/snapshot_restore_cluster_table.sql @@ -14,7 +14,7 @@ select * from mo_catalog.cluster_table_1; create snapshot snapshot1 for account acc01; -- @session -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; drop table if exists cluster_table_1; diff --git a/test/distributed/cases/snapshot/snapshot_restore_database_level.result b/test/distributed/cases/snapshot/snapshot_restore_database_level.result index f8a8a4f40b9e1..ee60fd3b1b1c5 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_database_level.result +++ b/test/distributed/cases/snapshot/snapshot_restore_database_level.result @@ -20,7 +20,7 @@ select count(*) from snapshot_read.users; count(*) 11 drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; delete from test_snapshot_read where a <= 50; DELETE FROM users where email = 'john@example.com'; UPDATE users SET password = 'newsecurepassword123' WHERE email = 'alice.jones@gmail.com'; @@ -62,7 +62,7 @@ select count(*) from snapshot_read.users; count(*) 11 drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; drop table users; drop table test_snapshot_read; select count(*) from snapshot_read.test_snapshot_read; @@ -100,7 +100,7 @@ select count(*) from snapshot_read.users; count(*) 11 drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; drop database snapshot_read; select count(*) from snapshot_read.test_snapshot_read; Unknown database snapshot_read diff --git a/test/distributed/cases/snapshot/snapshot_restore_database_level.sql b/test/distributed/cases/snapshot/snapshot_restore_database_level.sql index 21112741e085d..88bc058cae717 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_database_level.sql +++ b/test/distributed/cases/snapshot/snapshot_restore_database_level.sql @@ -18,7 +18,7 @@ INSERT INTO users (username, email, password) VALUES ('emily_adams', 'emily.adam select count(*) from snapshot_read.test_snapshot_read; select count(*) from snapshot_read.users; drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; delete from test_snapshot_read where a <= 50; DELETE FROM users where email = 'john@example.com'; UPDATE users SET password = 'newsecurepassword123' WHERE email = 'alice.jones@gmail.com'; @@ -53,7 +53,7 @@ INSERT INTO users (username, email, password) VALUES ('emily_adams', 'emily.adam select count(*) from snapshot_read.test_snapshot_read; select count(*) from snapshot_read.users; drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; drop table users; drop table test_snapshot_read; @@ -92,7 +92,7 @@ INSERT INTO users (username, email, password) VALUES ('emily_adams', 'emily.adam select count(*) from snapshot_read.test_snapshot_read; select count(*) from snapshot_read.users; drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; drop database snapshot_read; diff --git a/test/distributed/cases/snapshot/snapshot_restore_keyword.result b/test/distributed/cases/snapshot/snapshot_restore_keyword.result index 911434548d7f8..7f186e2cd76aa 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_keyword.result +++ b/test/distributed/cases/snapshot/snapshot_restore_keyword.result @@ -28,7 +28,7 @@ show create table `_binary`; Table Create Table _binary CREATE TABLE `_binary` (\n `add` int DEFAULT NULL,\n `all` bigint DEFAULT NULL,\n `alter` smallint DEFAULT NULL,\n `analyze` decimal(38,0) DEFAULT NULL,\n `and` char(1) DEFAULT NULL,\n `as` varchar(65535) DEFAULT NULL,\n `asc` int DEFAULT NULL,\n `begin` float DEFAULT NULL\n) drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; drop table `_binary`; restore account sys from snapshot sp01; show databases; @@ -61,7 +61,7 @@ Table Create Table current_date CREATE TABLE `current_date` (\n `current_role` int DEFAULT NULL,\n `current_time` bigint DEFAULT NULL,\n `current_timestamp` smallint DEFAULT NULL,\n `current_user` decimal(38,0) DEFAULT NULL,\n `database` char(1) DEFAULT NULL,\n `databases` varchar(65535) DEFAULT NULL,\n `day_hour` int DEFAULT NULL,\n `day_microsecond` float DEFAULT NULL\n) drop table `current_date`; drop snapshot if exists `div`; -create snapshot `div` for account sys; +create snapshot `div` for account; drop database `column`; restore account sys from snapshot `div`; use `column`; diff --git a/test/distributed/cases/snapshot/snapshot_restore_keyword.sql b/test/distributed/cases/snapshot/snapshot_restore_keyword.sql index e3c5858be1acf..b2aa5c4d432bd 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_keyword.sql +++ b/test/distributed/cases/snapshot/snapshot_restore_keyword.sql @@ -41,7 +41,7 @@ create table `_binary` (`add` int, `all` bigint, `alter` smallint, `analyze` dec show create table `_binary`; drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; drop table `_binary`; restore account sys from snapshot sp01; @@ -66,7 +66,7 @@ show create table `current_date`; drop table `current_date`; drop snapshot if exists `div`; -create snapshot `div` for account sys; +create snapshot `div` for account; drop database `column`; restore account sys from snapshot `div`; diff --git a/test/distributed/cases/snapshot/snapshot_restore_publication.result b/test/distributed/cases/snapshot/snapshot_restore_publication.result index 24b7cf84ec1b3..0627f4d11c3da 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_publication.result +++ b/test/distributed/cases/snapshot/snapshot_restore_publication.result @@ -7,7 +7,7 @@ create account test_tenant_1 admin_name 'test_account' identified by '111'; create publication pubname1 database db1 account test_tenant_1 comment 'publish db1 database'; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pubname1 db1 * test_tenant_1 2024-08-01 17:01:13 null publish db1 database +pubname1 db1 * test_tenant_1 2024-12-31 11:29:13 null publish db1 database show databases like 'db1'; Database db1 @@ -29,7 +29,7 @@ a 1 2 3 -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; drop publication pubname1; drop database db1; show publications; @@ -120,8 +120,8 @@ a 3 show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pubname2 sys db2 * publish db2 database 2024-08-01 17:01:15 sub_db2 2024-08-01 17:01:15 0 -create snapshot snapshot3 for account sys; +pubname2 sys db2 * publish db2 database 2024-12-31 11:29:15 sub_db2 2024-12-31 11:29:15 0 +create snapshot snapshot3 for account; drop publication pubname1; drop publication pubname2; drop database db1; @@ -204,7 +204,7 @@ show publications; publication database tables sub_account subscribed_accounts create_time update_time comments show databases like 'db%'; Database -create snapshot snapshot4 for account sys; +create snapshot snapshot4 for account; create database db1; use db1; create table t1(a int); @@ -308,7 +308,7 @@ show databases like 'db%'; Database show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -create snapshot snapshot5 for account sys; +create snapshot snapshot5 for account; create database db1; use db1; create table t1(a int); @@ -327,8 +327,8 @@ Tables_in_sub_db1 t1 show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pubname1 sys db1 * publish db1 database 2024-08-01 17:01:18 sub_db1 2024-08-01 17:01:18 0 -create snapshot snapshot6 for account sys; +pubname1 sys db1 * publish db1 database 2024-12-31 11:29:29 sub_db1 2024-12-31 11:29:29 0 +create snapshot snapshot6 for account; drop publication pubname1; drop database db1; create database db2; diff --git a/test/distributed/cases/snapshot/snapshot_restore_publication.sql b/test/distributed/cases/snapshot/snapshot_restore_publication.sql index 0d5bc7eb4a191..820752795b8cd 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_publication.sql +++ b/test/distributed/cases/snapshot/snapshot_restore_publication.sql @@ -25,7 +25,7 @@ show tables; select * from t1; -- @session -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; drop publication pubname1; drop database db1; @@ -112,7 +112,7 @@ select * from t2; show subscriptions; -- @session -create snapshot snapshot3 for account sys; +create snapshot snapshot3 for account; drop publication pubname1; drop publication pubname2; @@ -182,7 +182,7 @@ show databases like 'db%'; -- create and then restore -create snapshot snapshot4 for account sys; +create snapshot snapshot4 for account; create database db1; use db1; create table t1(a int); @@ -275,7 +275,7 @@ show databases like 'db%'; show snapshots; -- create, drop and create, then restore -create snapshot snapshot5 for account sys; +create snapshot snapshot5 for account; create database db1; use db1; create table t1(a int); @@ -301,7 +301,7 @@ show subscriptions; -- @session -create snapshot snapshot6 for account sys; +create snapshot snapshot6 for account; drop publication pubname1; drop database db1; diff --git a/test/distributed/cases/snapshot/snapshot_restore_system_table.result b/test/distributed/cases/snapshot/snapshot_restore_system_table.result index e17451f4f0f21..d3373ffceb886 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_system_table.result +++ b/test/distributed/cases/snapshot/snapshot_restore_system_table.result @@ -12,15 +12,15 @@ select name, db from mo_catalog.mo_user_defined_function; name db helloworld db1 helloworld5 db1 -create snapshot snapshot1 for account sys; +create snapshot snapshot1 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot1 2024-11-14 06:43:07.253292411 account sys +snapshot1 2024-12-31 03:23:52.614646 account sys drop function helloworld(); select name, db from mo_catalog.mo_user_defined_function; name db helloworld5 db1 -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME snapshot2 2024-11-14 06:43:07.264401704 account sys @@ -71,10 +71,10 @@ my_ext_stage disabled my_ext_stage1 disabled my_ext_stage2 disabled my_ext_stage3 enabled -create snapshot snapshot1 for account sys; +create snapshot snapshot1 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot1 2024-11-14 06:43:07.670395025 account sys +snapshot1 2024-12-31 03:23:53.76652 account sys ALTER STAGE my_ext_stage4 SET URL='s3://load/files2/'; internal error: the stage my_ext_stage4 not exists ALTER STAGE if exists my_ext_stage4 SET URL='s3://load/files2/'; @@ -82,11 +82,11 @@ ALTER STAGE my_ext_stage1 SET URL='s3://load/files2/' CREDENTIALS={'AWS_KEY_ID'= internal error: at most one option at a time ALTER STAGE my_ext_stage1 SET URL='s3://load/files2/'; ALTER STAGE my_ext_stage1 SET CREDENTIALS={'AWS_KEY_ID'='1a2b3d' ,'AWS_SECRET_KEY'='4x5y6z'}; -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:07.686982237 account sys -snapshot1 2024-11-14 06:43:07.670395025 account sys +snapshot2 2024-12-31 03:23:53.875709 account sys +snapshot1 2024-12-31 03:23:53.76652 account sys DROP STAGE my_ext_stage5; internal error: the stage my_ext_stage5 not exists DROP STAGE if exists my_ext_stage5; @@ -97,8 +97,8 @@ DROP STAGE my_ext_stage3; restore account sys from snapshot snapshot1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:07.686982237 account sys -snapshot1 2024-11-14 06:43:07.670395025 account sys +snapshot2 2024-12-31 03:23:53.875709 account sys +snapshot1 2024-12-31 03:23:53.76652 account sys SELECT stage_name from mo_catalog.mo_stages; stage_name my_ext_stage @@ -108,8 +108,8 @@ my_ext_stage3 restore account sys from snapshot snapshot2; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:07.686982237 account sys -snapshot1 2024-11-14 06:43:07.670395025 account sys +snapshot2 2024-12-31 03:23:53.875709 account sys +snapshot1 2024-12-31 03:23:53.76652 account sys SELECT stage_name from mo_catalog.mo_stages; stage_name my_ext_stage @@ -149,10 +149,10 @@ id val select name from mo_catalog.mo_stored_procedure; name test_if_hit_if -create snapshot snapshot1 for account sys; +create snapshot snapshot1 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot1 2024-11-14 06:43:08.205829849 account sys +snapshot1 2024-12-31 03:23:55.377673 account sys drop procedure if exists test_if_hit_if; drop procedure if exists test_if_hit_elseif_first_elseif; create procedure test_if_hit_elseif_first_elseif() 'begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end' @@ -164,16 +164,16 @@ id val select name from mo_catalog.mo_stored_procedure; name test_if_hit_elseif_first_elseif -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:08.22963861 account sys -snapshot1 2024-11-14 06:43:08.205829849 account sys +snapshot2 2024-12-31 03:23:55.509573 account sys +snapshot1 2024-12-31 03:23:55.377673 account sys restore account sys from snapshot snapshot1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:08.22963861 account sys -snapshot1 2024-11-14 06:43:08.205829849 account sys +snapshot2 2024-12-31 03:23:55.509573 account sys +snapshot1 2024-12-31 03:23:55.377673 account sys select name from mo_catalog.mo_stored_procedure; name test_if_hit_if @@ -185,8 +185,8 @@ id val restore account sys from snapshot snapshot2; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:08.22963861 account sys -snapshot1 2024-11-14 06:43:08.205829849 account sys +snapshot2 2024-12-31 03:23:55.509573 account sys +snapshot1 2024-12-31 03:23:55.377673 account sys select name from mo_catalog.mo_stored_procedure; name test_if_hit_elseif_first_elseif @@ -232,10 +232,10 @@ role_id user_id 1 3 1 4 1 5 -create snapshot snapshot1 for account sys; +create snapshot snapshot1 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot1 2024-11-14 06:43:08.710821386 account sys +snapshot1 2024-12-31 03:23:56.520966 account sys drop user if exists u_a, u_b, u_d; alter user u_c identified by 'c111111'; select user_id,user_name,authentication_string from mo_catalog.mo_user; @@ -250,16 +250,16 @@ role_id user_id 0 1 1 1 1 4 -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:08.733580409 account sys -snapshot1 2024-11-14 06:43:08.710821386 account sys +snapshot2 2024-12-31 03:23:56.664168 account sys +snapshot1 2024-12-31 03:23:56.520966 account sys restore account sys from snapshot snapshot1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:08.733580409 account sys -snapshot1 2024-11-14 06:43:08.710821386 account sys +snapshot2 2024-12-31 03:23:56.664168 account sys +snapshot1 2024-12-31 03:23:56.520966 account sys select user_id,user_name,authentication_string from mo_catalog.mo_user; user_id user_name authentication_string 0 root *832EB84CB764129D05D498ED9CA7E5CE9B8F83EB @@ -281,8 +281,8 @@ role_id user_id restore account sys from snapshot snapshot2; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:08.733580409 account sys -snapshot1 2024-11-14 06:43:08.710821386 account sys +snapshot2 2024-12-31 03:23:56.664168 account sys +snapshot1 2024-12-31 03:23:56.520966 account sys select user_id,user_name,authentication_string from mo_catalog.mo_user; user_id user_name authentication_string 0 root *832EB84CB764129D05D498ED9CA7E5CE9B8F83EB @@ -328,7 +328,7 @@ helloworld5 db1 create snapshot snapshot1 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot1 2024-11-14 06:43:09.497478675 account acc01 +snapshot1 2024-12-31 03:24:12.455419 account acc01 drop function helloworld(); select name, db from mo_catalog.mo_user_defined_function; name db @@ -336,16 +336,16 @@ helloworld5 db1 create snapshot snapshot2 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:09.532578705 account acc01 -snapshot1 2024-11-14 06:43:09.497478675 account acc01 +snapshot2 2024-12-31 03:24:13.195737 account acc01 +snapshot1 2024-12-31 03:24:12.455419 account acc01 drop database db1; select name, db from mo_catalog.mo_user_defined_function; name db restore account acc01 from snapshot snapshot1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:09.532578705 account acc01 -snapshot1 2024-11-14 06:43:09.497478675 account acc01 +snapshot2 2024-12-31 03:24:13.195737 account acc01 +snapshot1 2024-12-31 03:24:12.455419 account acc01 select name, db from mo_catalog.mo_user_defined_function; name db helloworld db1 @@ -353,8 +353,8 @@ helloworld5 db1 restore account acc01 from snapshot snapshot2; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:09.532578705 account acc01 -snapshot1 2024-11-14 06:43:09.497478675 account acc01 +snapshot2 2024-12-31 03:24:13.195737 account acc01 +snapshot1 2024-12-31 03:24:12.455419 account acc01 select name, db from mo_catalog.mo_user_defined_function; name db helloworld5 db1 @@ -387,7 +387,7 @@ my_ext_stage3 enabled create snapshot snapshot1 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot1 2024-11-14 06:43:09.886334552 account acc01 +snapshot1 2024-12-31 03:24:14.851876 account acc01 ALTER STAGE my_ext_stage4 SET URL='s3://load/files2/'; internal error: the stage my_ext_stage4 not exists ALTER STAGE if exists my_ext_stage4 SET URL='s3://load/files2/'; @@ -398,8 +398,8 @@ ALTER STAGE my_ext_stage1 SET CREDENTIALS={'AWS_KEY_ID'='1a2b3d' ,'AWS_SECRET_KE create snapshot snapshot2 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:09.903535237 account acc01 -snapshot1 2024-11-14 06:43:09.886334552 account acc01 +snapshot2 2024-12-31 03:24:14.954756 account acc01 +snapshot1 2024-12-31 03:24:14.851876 account acc01 DROP STAGE my_ext_stage5; internal error: the stage my_ext_stage5 not exists DROP STAGE if exists my_ext_stage5; @@ -410,8 +410,8 @@ DROP STAGE my_ext_stage3; restore account acc01 from snapshot snapshot1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:09.903535237 account acc01 -snapshot1 2024-11-14 06:43:09.886334552 account acc01 +snapshot2 2024-12-31 03:24:14.954756 account acc01 +snapshot1 2024-12-31 03:24:14.851876 account acc01 SELECT stage_name from mo_catalog.mo_stages; stage_name my_ext_stage @@ -421,8 +421,8 @@ my_ext_stage3 restore account acc01 from snapshot snapshot2; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:09.903535237 account acc01 -snapshot1 2024-11-14 06:43:09.886334552 account acc01 +snapshot2 2024-12-31 03:24:14.954756 account acc01 +snapshot1 2024-12-31 03:24:14.851876 account acc01 SELECT stage_name from mo_catalog.mo_stages; stage_name my_ext_stage @@ -465,7 +465,7 @@ test_if_hit_if create snapshot snapshot1 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot1 2024-11-14 06:43:10.342839922 account acc01 +snapshot1 2024-12-31 03:24:16.348045 account acc01 drop procedure if exists test_if_hit_if; drop procedure if exists test_if_hit_elseif_first_elseif; create procedure test_if_hit_elseif_first_elseif() 'begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end' @@ -480,13 +480,13 @@ test_if_hit_elseif_first_elseif create snapshot snapshot2 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:10.381886724 account acc01 -snapshot1 2024-11-14 06:43:10.342839922 account acc01 +snapshot2 2024-12-31 03:24:16.487551 account acc01 +snapshot1 2024-12-31 03:24:16.348045 account acc01 restore account acc01 from snapshot snapshot1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:10.381886724 account acc01 -snapshot1 2024-11-14 06:43:10.342839922 account acc01 +snapshot2 2024-12-31 03:24:16.487551 account acc01 +snapshot1 2024-12-31 03:24:16.348045 account acc01 select name from mo_catalog.mo_stored_procedure; name test_if_hit_if @@ -498,8 +498,8 @@ id val restore account acc01 from snapshot snapshot2; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:10.381886724 account acc01 -snapshot1 2024-11-14 06:43:10.342839922 account acc01 +snapshot2 2024-12-31 03:24:16.487551 account acc01 +snapshot1 2024-12-31 03:24:16.348045 account acc01 select name from mo_catalog.mo_stored_procedure; name test_if_hit_elseif_first_elseif @@ -528,7 +528,7 @@ u_d *97D6A234F2882F45432E3C14150E6B9ACAB2255F create snapshot snapshot1 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot1 2024-11-14 06:43:10.830651105 account acc01 +snapshot1 2024-12-31 03:24:17.119166 account acc01 drop user if exists u_a, u_b, u_d; alter user u_c identified by 'c111111'; select user_name,authentication_string from mo_catalog.mo_user; @@ -538,13 +538,13 @@ u_c *7F494961F862DCB97144C0E5D77CE4FCD30D8281 create snapshot snapshot2 for account acc01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:10.850824054 account acc01 -snapshot1 2024-11-14 06:43:10.830651105 account acc01 +snapshot2 2024-12-31 03:24:17.212708 account acc01 +snapshot1 2024-12-31 03:24:17.119166 account acc01 restore account acc01 from snapshot snapshot1; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:10.850824054 account acc01 -snapshot1 2024-11-14 06:43:10.830651105 account acc01 +snapshot2 2024-12-31 03:24:17.212708 account acc01 +snapshot1 2024-12-31 03:24:17.119166 account acc01 select user_name,authentication_string from mo_catalog.mo_user; user_name authentication_string test_account *832EB84CB764129D05D498ED9CA7E5CE9B8F83EB @@ -555,8 +555,8 @@ u_d *97D6A234F2882F45432E3C14150E6B9ACAB2255F restore account acc01 from snapshot snapshot2; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -snapshot2 2024-11-14 06:43:10.850824054 account acc01 -snapshot1 2024-11-14 06:43:10.830651105 account acc01 +snapshot2 2024-12-31 03:24:17.212708 account acc01 +snapshot1 2024-12-31 03:24:17.119166 account acc01 select user_name,authentication_string from mo_catalog.mo_user; user_name authentication_string test_account *832EB84CB764129D05D498ED9CA7E5CE9B8F83EB @@ -745,7 +745,7 @@ a b account_id 1 1 0 0 0 1 1 1 1 -create snapshot snapshot1 for account sys; +create snapshot snapshot1 for account; drop table if exists cluster_table_1; create cluster table cluster_table_2(a int, b int); insert into cluster_table_2 values(0,0,0),(1,1,0); @@ -756,7 +756,7 @@ a b account_id 1 1 0 0 0 1 1 1 1 -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; restore account sys from snapshot snapshot1; select * from mo_catalog.cluster_table_1; a b account_id diff --git a/test/distributed/cases/snapshot/snapshot_restore_system_table.sql b/test/distributed/cases/snapshot/snapshot_restore_system_table.sql index 66ecd24e48023..ea4354548e96c 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_system_table.sql +++ b/test/distributed/cases/snapshot/snapshot_restore_system_table.sql @@ -9,14 +9,14 @@ select name, db from mo_catalog.mo_user_defined_function; create function db1.helloworld5 () returns int language sql as 'select id from tbl1 limit 1'; select name, db from mo_catalog.mo_user_defined_function; -create snapshot snapshot1 for account sys; +create snapshot snapshot1 for account; -- @ignore:1 show snapshots; drop function helloworld(); select name, db from mo_catalog.mo_user_defined_function; -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; -- @ignore:1 show snapshots; @@ -53,7 +53,7 @@ SELECT stage_name, stage_status from mo_catalog.mo_stages; CREATE STAGE my_ext_stage3 URL='s3://load/files/' CREDENTIALS={'AWS_KEY_ID'='1a2b3c' ,'AWS_SECRET_KEY'='4x5y6z'} ENABLE = TRUE; SELECT stage_name, stage_status from mo_catalog.mo_stages; -create snapshot snapshot1 for account sys; +create snapshot snapshot1 for account; -- @ignore:1 show snapshots; @@ -63,7 +63,7 @@ ALTER STAGE my_ext_stage1 SET URL='s3://load/files2/' CREDENTIALS={'AWS_KEY_ID'= ALTER STAGE my_ext_stage1 SET URL='s3://load/files2/'; ALTER STAGE my_ext_stage1 SET CREDENTIALS={'AWS_KEY_ID'='1a2b3d' ,'AWS_SECRET_KEY'='4x5y6z'}; -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; -- @ignore:1 show snapshots; @@ -122,7 +122,7 @@ call test_if_hit_if(); select name from mo_catalog.mo_stored_procedure; -create snapshot snapshot1 for account sys; +create snapshot snapshot1 for account; -- @ignore:1 show snapshots; @@ -135,7 +135,7 @@ create procedure test_if_hit_elseif_first_elseif() 'begin DECLARE v1 INT; SET v1 call test_if_hit_elseif_first_elseif(); select name from mo_catalog.mo_stored_procedure; -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; -- @ignore:1 show snapshots; @@ -173,7 +173,7 @@ select user_id,user_name,authentication_string from mo_catalog.mo_user; -- @ignore:0,1 select role_id,user_id from mo_catalog.mo_user_grant; -create snapshot snapshot1 for account sys; +create snapshot snapshot1 for account; -- @ignore:1 show snapshots; drop user if exists u_a, u_b, u_d; @@ -184,7 +184,7 @@ select user_id,user_name,authentication_string from mo_catalog.mo_user; -- @ignore:0,1 select role_id,user_id from mo_catalog.mo_user_grant; -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; -- @ignore:1 show snapshots; @@ -593,7 +593,7 @@ insert into cluster_table_1 values(0,0,0),(1,1,0); insert into cluster_table_1 values(0,0,1),(1,1,1); select * from cluster_table_1; -create snapshot snapshot1 for account sys; +create snapshot snapshot1 for account; drop table if exists cluster_table_1; create cluster table cluster_table_2(a int, b int); @@ -601,7 +601,7 @@ insert into cluster_table_2 values(0,0,0),(1,1,0); insert into cluster_table_2 values(0,0,1),(1,1,1); select * from cluster_table_2; -create snapshot snapshot2 for account sys; +create snapshot snapshot2 for account; restore account sys from snapshot snapshot1; select * from mo_catalog.cluster_table_1; diff --git a/test/distributed/cases/snapshot/snapshot_restore_table_level.result b/test/distributed/cases/snapshot/snapshot_restore_table_level.result index 7d98c12dcc02f..1b6ad03b45fed 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_table_level.result +++ b/test/distributed/cases/snapshot/snapshot_restore_table_level.result @@ -5,7 +5,7 @@ INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), select count(*) from snapshot_read.test_snapshot_read; count(*) 100 -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; delete from test_snapshot_read where a <= 50; select count(*) from snapshot_read.test_snapshot_read; count(*) @@ -13,7 +13,7 @@ count(*) select count(*) from snapshot_read.test_snapshot_read {snapshot = 'snapshot_01'}; count(*) 100 -create snapshot snapshot_02 for account sys; +create snapshot snapshot_02 for account; INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40); select count(*) from snapshot_read.test_snapshot_read; count(*) @@ -51,7 +51,7 @@ select id, username, email from snapshot_read.users where email = 'alice.jones@g id username email 3 alice_jones alice.jones@gmail.com drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; DELETE FROM users where email = 'john@example.com'; UPDATE users SET password = 'newsecurepassword123' WHERE email = 'alice.jones@gmail.com'; select id, username, email from snapshot_read.users where email = 'john@example.com'; @@ -97,7 +97,7 @@ select count(*) from snapshot_read.users; count(*) 11 drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; drop table users; select count(*) from snapshot_read.users; SQL parser error: table "users" does not exist @@ -123,7 +123,7 @@ select count(*) from snapshot_read.users; count(*) 11 drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; drop database snapshot_read; select count(*) from snapshot_read.users; Unknown database snapshot_read @@ -326,7 +326,7 @@ select count(*) from snapshot_read.users; count(*) 11 drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; delete from test_snapshot_read where a <= 50; select count(*) from snapshot_read.test_snapshot_read; count(*) diff --git a/test/distributed/cases/snapshot/snapshot_restore_table_level.sql b/test/distributed/cases/snapshot/snapshot_restore_table_level.sql index e31136550ec6b..4b4a3f8faddd7 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_table_level.sql +++ b/test/distributed/cases/snapshot/snapshot_restore_table_level.sql @@ -4,11 +4,11 @@ use snapshot_read; create table test_snapshot_read (a int); INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40),(41), (42), (43), (44), (45), (46), (47), (48), (49), (50),(51), (52), (53), (54), (55), (56), (57), (58), (59), (60),(61), (62), (63), (64), (65), (66), (67), (68), (69), (70),(71), (72), (73), (74), (75), (76), (77), (78), (79), (80), (81), (82), (83), (84), (85), (86), (87), (88), (89), (90),(91), (92), (93), (94), (95), (96), (97), (98), (99), (100); select count(*) from snapshot_read.test_snapshot_read; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; delete from test_snapshot_read where a <= 50; select count(*) from snapshot_read.test_snapshot_read; select count(*) from snapshot_read.test_snapshot_read {snapshot = 'snapshot_01'}; -create snapshot snapshot_02 for account sys; +create snapshot snapshot_02 for account; INSERT INTO test_snapshot_read (a) VALUES(1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40); select count(*) from snapshot_read.test_snapshot_read; select count(*) from snapshot_read.test_snapshot_read{snapshot = 'snapshot_02'}; @@ -38,7 +38,7 @@ INSERT INTO users (username, email, password) VALUES ('emily_adams', 'emily.adam select id, username, email from snapshot_read.users where email = 'john@example.com'; select id, username, email from snapshot_read.users where email = 'alice.jones@gmail.com'; drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; DELETE FROM users where email = 'john@example.com'; UPDATE users SET password = 'newsecurepassword123' WHERE email = 'alice.jones@gmail.com'; @@ -83,7 +83,7 @@ INSERT INTO users (username, email, password) VALUES ('emily_adams', 'emily.adam select count(*) from snapshot_read.users; drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; drop table users; select count(*) from snapshot_read.users; restore account sys database snapshot_read table users from snapshot sp_01; @@ -107,7 +107,7 @@ INSERT INTO users (username, email, password) VALUES ('emily_adams', 'emily.adam select count(*) from snapshot_read.users; drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; drop database snapshot_read; select count(*) from snapshot_read.users; restore account sys database snapshot_read table users from snapshot sp_01; @@ -297,7 +297,7 @@ INSERT INTO users (username, email, password) VALUES ('emily_adams', 'emily.adam select count(*) from snapshot_read.users; drop snapshot if exists sp_01; -create snapshot sp_01 for account sys; +create snapshot sp_01 for account; delete from test_snapshot_read where a <= 50; select count(*) from snapshot_read.test_snapshot_read; diff --git a/test/distributed/cases/snapshot/snapshot_restore_to_table_fk.result b/test/distributed/cases/snapshot/snapshot_restore_to_table_fk.result index 0c3e10388f4b1..5a5bb7616a237 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_to_table_fk.result +++ b/test/distributed/cases/snapshot/snapshot_restore_to_table_fk.result @@ -413,7 +413,7 @@ departmentid budget drop snapshot if exists snapshot_01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; Drop database Payroll; Drop database Projects; Drop database Company; diff --git a/test/distributed/cases/snapshot/snapshot_restore_to_table_fk.sql b/test/distributed/cases/snapshot/snapshot_restore_to_table_fk.sql index 448d206b50629..bacc3848ec1b9 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_to_table_fk.sql +++ b/test/distributed/cases/snapshot/snapshot_restore_to_table_fk.sql @@ -348,7 +348,7 @@ select * from DepartmentBudgets; drop snapshot if exists snapshot_01; -- @ignore:1 show snapshots; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; Drop database Payroll; Drop database Projects; diff --git a/test/distributed/cases/snapshot/snapshot_restore_to_table_fk_2.result b/test/distributed/cases/snapshot/snapshot_restore_to_table_fk_2.result index 2943b7cfe1baf..222b3da0056f3 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_to_table_fk_2.result +++ b/test/distributed/cases/snapshot/snapshot_restore_to_table_fk_2.result @@ -316,7 +316,7 @@ departmentid budget drop snapshot if exists snapshot_01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; Drop database Payroll; Drop database Projects; Drop database Company; @@ -755,7 +755,7 @@ departmentid budget drop snapshot if exists snapshot_01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; Drop database Payroll; Drop database Projects; Drop database Company; @@ -920,7 +920,7 @@ count(*) drop snapshot if exists snapshot_01; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; DROP DATABASE IF EXISTS Company; CREATE DATABASE Company; USE Company; diff --git a/test/distributed/cases/snapshot/snapshot_restore_to_table_fk_2.sql b/test/distributed/cases/snapshot/snapshot_restore_to_table_fk_2.sql index f27085a0a42b3..fd48a3296deee 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_to_table_fk_2.sql +++ b/test/distributed/cases/snapshot/snapshot_restore_to_table_fk_2.sql @@ -254,7 +254,7 @@ select * from DepartmentBudgets; drop snapshot if exists snapshot_01; -- @ignore:1 show snapshots; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; Drop database Payroll; Drop database Projects; @@ -540,7 +540,7 @@ select * from DepartmentBudgets; drop snapshot if exists snapshot_01; -- @ignore:1 show snapshots; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; Drop database Payroll; Drop database Projects; @@ -615,7 +615,7 @@ select count(*) from aff01; drop snapshot if exists snapshot_01; -- @ignore:1 show snapshots; -create snapshot snapshot_01 for account sys; +create snapshot snapshot_01 for account; DROP DATABASE IF EXISTS Company; CREATE DATABASE Company; diff --git a/test/distributed/cases/snapshot/snapshot_restore_to_table_view.result b/test/distributed/cases/snapshot/snapshot_restore_to_table_view.result index 0b372b9c61472..eab0faba60386 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_to_table_view.result +++ b/test/distributed/cases/snapshot/snapshot_restore_to_table_view.result @@ -30,7 +30,7 @@ employeeid employeename salary salaryrank 1 Alice 70000.00 4 4 David 65000.00 5 drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; drop database test01; restore account sys from snapshot sp06; select * from test01.EmployeeSalaries; @@ -81,7 +81,7 @@ employeeid employeename salary salaryrank 1 Alice 70000.00 4 4 David 65000.00 5 drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; drop table EmployeeSalaries; drop view EmployeeSalaryRanking; restore account sys from snapshot sp06; @@ -213,7 +213,7 @@ Human Resources 1000.00 0.00 drop snapshot if exists sp06; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -create snapshot sp06 for account sys; +create snapshot sp06 for account; drop database test01; restore account sys from snapshot sp06; select * from test01.employees; @@ -541,7 +541,7 @@ Bob Johnson 3.70 3.60000000 drop snapshot if exists sp06; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -create snapshot sp06 for account sys; +create snapshot sp06 for account; drop database School; drop database University; drop database EducationSystem; diff --git a/test/distributed/cases/snapshot/snapshot_restore_to_table_view.sql b/test/distributed/cases/snapshot/snapshot_restore_to_table_view.sql index 386ce43f9d008..dafd2ad1eebb9 100644 --- a/test/distributed/cases/snapshot/snapshot_restore_to_table_view.sql +++ b/test/distributed/cases/snapshot/snapshot_restore_to_table_view.sql @@ -26,7 +26,7 @@ from select * from EmployeeSalaryRanking; drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; drop database test01; @@ -67,7 +67,7 @@ from select * from EmployeeSalaryRanking; drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; drop table EmployeeSalaries; drop view EmployeeSalaryRanking; @@ -179,7 +179,7 @@ SELECT * FROM combined_sales_view; drop snapshot if exists sp06; -- @ignore:1 show snapshots; -create snapshot sp06 for account sys; +create snapshot sp06 for account; drop database test01; @@ -435,7 +435,7 @@ select * from StudentOverallPerformance; drop snapshot if exists sp06; -- @ignore:1 show snapshots; -create snapshot sp06 for account sys; +create snapshot sp06 for account; drop database School; drop database University; diff --git a/test/distributed/cases/snapshot/sys_restore_pubsub_to_sys_account.result b/test/distributed/cases/snapshot/sys_restore_pubsub_to_sys_account.result index fdd0da8f2a571..c018bea0b4252 100644 --- a/test/distributed/cases/snapshot/sys_restore_pubsub_to_sys_account.result +++ b/test/distributed/cases/snapshot/sys_restore_pubsub_to_sys_account.result @@ -8,20 +8,20 @@ insert into repub01 values (1); insert into repub01 values (2); insert into repub01 (col1) values (3); drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; drop publication if exists publication01; create publication publication01 database republication01 account acc01 comment 'publish before creating snapshot'; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -publication01 republication01 * acc01 2024-08-03 15:32:19 null publish before creating snapshot +publication01 republication01 * acc01 2024-12-31 11:10:09 null publish before creating snapshot select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment -0 publication01 republication01 279336 true * acc01 2024-11-14 18:53:39 null 0 1 publish before creating snapshot +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +0 sys publication01 republication01 272592 true * acc01 2025-01-03 12:59:36 null 0 1 publish before creating snapshot drop publication publication01; drop database republication01; restore account sys from snapshot sp01; select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment show databases; Database information_schema @@ -76,15 +76,15 @@ insert into aff01 values (7521,'WARD','SALESMAN',7698,'1981-02-22',1250,500,30); insert into aff01 values (7566,'JONES','MANAGER',7839,'1981-04-02',2975,NULL,20); insert into aff01 values (7654,'MARTIN','SALESMAN',7698,'1981-09-28',1250,1400,30); drop snapshot if exists sp02; -create snapshot sp02 for account sys; +create snapshot sp02 for account; drop publication if exists pub02; create publication pub02 database repub02 account acc01 comment 'publish before creating snapshot'; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub02 repub02 * acc01 2024-08-03 15:32:19 null publish before creating snapshot +pub02 repub02 * acc01 2024-12-31 11:10:10 null publish before creating snapshot select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment -0 pub02 repub02 279361 true * acc01 2024-11-14 18:53:40 null 0 1 publish before creating snapshot +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +0 sys pub02 repub02 272618 true * acc01 2025-01-03 12:59:37 null 0 1 publish before creating snapshot drop database if exists sub01; create database sub01 from sys publication pub02; show databases; @@ -117,7 +117,7 @@ restore account sys from snapshot sp02; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment show databases; Database information_schema @@ -181,14 +181,14 @@ drop publication if exists pub03; create publication pub03 database repub03 account acc01 comment 'create repub03'; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub03 repub03 * acc01 2024-08-03 15:32:19 null create repub03 +pub03 repub03 * acc01 2024-12-31 11:10:12 null create repub03 drop snapshot if exists sp01; create snapshot sp01 for account acc01; drop database if exists sub03; create database sub03 from sys publication pub03; show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub03 sys repub03 * create repub03 2024-08-03 15:32:19 sub03 2024-08-03 15:32:20 0 +pub03 sys repub03 * create repub03 2024-12-31 11:10:12 sub03 2024-12-31 11:10:14 0 show databases; Database information_schema @@ -230,8 +230,8 @@ system system_metrics drop snapshot sp01; select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment -0 pub03 repub03 279390 true * acc01 2024-11-14 18:53:41 null 0 1 create repub03 +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +0 sys pub03 repub03 272647 true * acc01 2025-01-03 12:59:38 null 0 1 create repub03 show databases; Database information_schema @@ -245,7 +245,7 @@ system system_metrics show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub03 repub03 * acc01 2024-08-03 15:32:19 null create repub03 +pub03 repub03 * acc01 2024-12-31 11:10:12 null create repub03 drop publication pub03; drop database repub03; drop database if exists db01; @@ -263,11 +263,11 @@ create snapshot sp02 for account acc01; drop publication if exists pub03; create publication pub04 database db01 account acc01 comment 'create pub04'; select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment -0 pub04 db01 279408 true * acc01 2024-11-14 18:53:42 null 0 1 create pub04 +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +0 sys pub04 db01 272665 true * acc01 2025-01-03 12:59:39 null 0 1 create pub04 show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub04 db01 * acc01 2024-08-03 15:32:20 null create pub04 +pub04 db01 * acc01 2024-12-31 11:10:15 null create pub04 drop database if exists sub04; create database sub04 from sys publication pub04; use sub04; @@ -289,7 +289,7 @@ use sub04; Unknown database sub04 show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub04 db01 * acc01 2024-08-03 15:32:20 null create pub04 +pub04 db01 * acc01 2024-12-31 11:10:15 null create pub04 drop publication pub04; drop database db01; drop snapshot sp02; @@ -348,8 +348,8 @@ drop publication if exists pub06; create publication pub06 database db10 account acc02 comment 'publish db10'; show publications; publication database tables sub_account subscribed_accounts create_time update_time comments -pub06 db10 * acc02 2024-08-03 15:32:21 null publish db10 -pub05 db09 * acc02 2024-08-03 15:32:21 null publish db09 +pub06 db10 * acc02 2024-12-31 11:10:16 null publish db10 +pub05 db09 * acc02 2024-12-31 11:10:16 null publish db09 drop database if exists sub05; create database sub05 from acc01 publication pub05; show databases; @@ -373,7 +373,7 @@ show publications; publication database tables sub_account subscribed_accounts create_time update_time comments show subscriptions; pub_name pub_account pub_database pub_tables pub_comment pub_time sub_name sub_time status -pub05 acc01 null null null null sub05 2024-08-09 16:04:39 2 +pub05 acc01 null null null null sub05 2024-12-31 11:10:16 2 show databases; Database information_schema @@ -468,14 +468,14 @@ drop table if exists test01; create table test01 (a int); insert into test01 (a) values (1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40),(41), (42), (43), (44), (45), (46), (47), (48), (49), (50),(51), (52), (53), (54), (55), (56), (57), (58), (59), (60),(61), (62), (63), (64), (65), (66), (67), (68), (69), (70),(71), (72), (73), (74), (75), (76), (77), (78), (79), (80), (81), (82), (83), (84), (85), (86), (87), (88), (89), (90),(91), (92), (93), (94), (95), (96), (97), (98), (99), (100); drop snapshot if exists sp100; -create snapshot sp100 for account sys; +create snapshot sp100 for account; drop publication if exists pub07; create publication pub07 database test01 account acc01 comment 'publish test01'; select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment -0 pub07 test01 279564 true * acc01 2024-11-14 18:53:43 null 0 1 publish test01 +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +0 sys pub07 test01 272822 true * acc01 2025-01-03 12:59:40 null 0 1 publish test01 drop snapshot if exists sp101; -create snapshot sp101 for account sys; +create snapshot sp101 for account; drop database if exists sub07; create database sub07 from sys publication pub07; use sub07; @@ -495,7 +495,7 @@ use sub07; internal error: there is no publication pub07 restore account sys from snapshot sp101; select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment show databases; Database information_schema @@ -696,8 +696,8 @@ a b 3 2 restore account acc02 from snapshot sp105 to account acc03; select * from mo_catalog.mo_pubs; -account_id pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment -57 pub10 test04 279822 true * acc02 2024-11-14 18:53:48 null 2 2 publish test03 +account_id account_name pub_name database_name database_id all_table table_list account_list created_time update_time owner creator comment +1 acc01 pub10 test04 273081 true * acc02 2025-01-03 12:59:43 null 2 2 publish test03 show databases; Database information_schema diff --git a/test/distributed/cases/snapshot/sys_restore_pubsub_to_sys_account.sql b/test/distributed/cases/snapshot/sys_restore_pubsub_to_sys_account.sql index 1cd240678cde2..3f581bf2e066c 100644 --- a/test/distributed/cases/snapshot/sys_restore_pubsub_to_sys_account.sql +++ b/test/distributed/cases/snapshot/sys_restore_pubsub_to_sys_account.sql @@ -11,19 +11,19 @@ insert into repub01 values (2); insert into repub01 (col1) values (3); drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; -- @ignore:1 drop publication if exists publication01; create publication publication01 database republication01 account acc01 comment 'publish before creating snapshot'; -- @ignore:5,6 show publications; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; drop publication publication01; drop database republication01; restore account sys from snapshot sp01; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; show databases; use republication01; @@ -76,13 +76,13 @@ insert into aff01 values (7566,'JONES','MANAGER',7839,'1981-04-02',2975,NULL,20) insert into aff01 values (7654,'MARTIN','SALESMAN',7698,'1981-09-28',1250,1400,30); drop snapshot if exists sp02; -create snapshot sp02 for account sys; +create snapshot sp02 for account; drop publication if exists pub02; create publication pub02 database repub02 account acc01 comment 'publish before creating snapshot'; -- @ignore:5,6 show publications; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @session:id=1&user=acc01:test_account&password=111 @@ -98,7 +98,7 @@ select * from aff01; restore account sys from snapshot sp02; -- @ignore:5,6 show publications; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; show databases; use repub02; @@ -161,7 +161,7 @@ show subscriptions; show databases; drop snapshot sp01; -- @session --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; show databases; -- @ignore:5,6 @@ -190,7 +190,7 @@ create snapshot sp02 for account acc01; drop publication if exists pub03; create publication pub04 database db01 account acc01 comment 'create pub04'; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @ignore:5,6 show publications; @@ -336,7 +336,7 @@ create snapshot sp11 for account acc01; drop publication if exists pub06; create publication pub06 database db10 account acc02 comment 'publish db10'; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @session @@ -357,7 +357,7 @@ restore account acc01 from snapshot sp11; show publications; show databases; drop database db10; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @session @@ -382,15 +382,15 @@ create table test01 (a int); insert into test01 (a) values (1), (2), (3), (4), (5),(6), (7), (8), (9), (10), (11), (12),(13), (14), (15), (16), (17), (18), (19), (20),(21), (22), (23), (24), (25), (26), (27), (28), (29), (30),(31), (32), (33), (34), (35), (36), (37), (38), (39), (40),(41), (42), (43), (44), (45), (46), (47), (48), (49), (50),(51), (52), (53), (54), (55), (56), (57), (58), (59), (60),(61), (62), (63), (64), (65), (66), (67), (68), (69), (70),(71), (72), (73), (74), (75), (76), (77), (78), (79), (80), (81), (82), (83), (84), (85), (86), (87), (88), (89), (90),(91), (92), (93), (94), (95), (96), (97), (98), (99), (100); drop snapshot if exists sp100; -create snapshot sp100 for account sys; +create snapshot sp100 for account; drop publication if exists pub07; create publication pub07 database test01 account acc01 comment 'publish test01'; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; drop snapshot if exists sp101; -create snapshot sp101 for account sys; +create snapshot sp101 for account; -- @session:id=1&user=acc01:test_account&password=111 drop database if exists sub07; @@ -407,7 +407,7 @@ use sub07; -- @session restore account sys from snapshot sp101; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @session:id=1&user=acc01:test_account&password=111 @@ -571,7 +571,7 @@ create snapshot sp105 for account acc02; -- @session:id=1&user=acc01:test_account&password=111 drop publication if exists pub10; create publication pub10 database test04 account acc02 comment 'publish test03'; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @session @@ -584,7 +584,7 @@ select * from t1; -- @session restore account acc02 from snapshot sp105 to account acc03; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @session:id=4&user=acc03:test_account&password=111 @@ -594,7 +594,7 @@ select * from t1; -- @ignore:5,6 show publications; drop database sub09; --- @ignore:0,3,7 +-- @ignore:0,4,8,9 select * from mo_catalog.mo_pubs; -- @session diff --git a/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.result b/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.result index e14242f7b3876..01b3dcf65f6ef 100644 --- a/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.result +++ b/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.result @@ -14,10 +14,10 @@ select name, db from mo_catalog.mo_user_defined_function; name db addab udf_db drop snapshot if exists udf_dsp01; -create snapshot udf_dsp01 for account sys; +create snapshot udf_dsp01 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -udf_dsp01 2024-05-31 03:23:46.078875 account sys +udf_dsp01 2024-12-31 03:03:10.325715 account sys create function `concatenate`(str1 varchar(255), str2 varchar(255)) returns varchar(255) language sql as '$1 + $2'; @@ -25,11 +25,11 @@ select concatenate('Hello, ', 'World!'); concatenate(Hello, , World!) Hello, World! drop snapshot if exists udf_dsp02; -create snapshot udf_dsp02 for account sys; +create snapshot udf_dsp02 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -udf_dsp02 2024-05-31 03:23:46.138061 account sys -udf_dsp01 2024-05-31 03:23:46.078875 account sys +udf_dsp02 2024-12-31 03:03:10.421479 account sys +udf_dsp01 2024-12-31 03:03:10.325715 account sys drop database if exists udf_db2; create database udf_db2; use udf_db2; @@ -45,40 +45,40 @@ addab udf_db concatenate udf_db subab udf_db2 drop snapshot if exists udf_dsp03; -create snapshot udf_dsp03 for account sys; +create snapshot udf_dsp03 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -udf_dsp03 2024-05-31 03:23:46.287773 account sys -udf_dsp02 2024-05-31 03:23:46.138061 account sys -udf_dsp01 2024-05-31 03:23:46.078875 account sys +udf_dsp03 2024-12-31 03:03:10.602564 account sys +udf_dsp02 2024-12-31 03:03:10.421479 account sys +udf_dsp01 2024-12-31 03:03:10.325715 account sys select * from mo_catalog.mo_user_defined_function; function_id name owner args rettype body language db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -1 addab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db dump 2024-05-31 03:23:46 2024-05-31 03:23:46 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci -2 concatenate 0 [{"name": "str1", "type": "varchar"}, {"name": "str2", "type": "varchar"}] varchar $1 + $2 sql udf_db dump 2024-05-31 03:23:46 2024-05-31 03:23:46 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci -3 subab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 - $2 sql udf_db2 dump 2024-05-31 03:23:46 2024-05-31 03:23:46 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +1 addab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db dump 2024-12-31 03:03:10 2024-12-31 03:03:10 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +2 concatenate 0 [{"name": "str1", "type": "varchar"}, {"name": "str2", "type": "varchar"}] varchar $1 + $2 sql udf_db dump 2024-12-31 03:03:10 2024-12-31 03:03:10 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 subab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 - $2 sql udf_db2 dump 2024-12-31 03:03:10 2024-12-31 03:03:10 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop function subab(x int,y int); drop function udf_db.concatenate(str1 varchar(255), str2 varchar(255)); restore account sys from snapshot udf_dsp03; select * from mo_catalog.mo_user_defined_function; function_id name owner args rettype body language db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -1 addab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db dump 2024-05-31 03:23:46 2024-05-31 03:23:46 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci -2 concatenate 0 [{"name": "str1", "type": "varchar"}, {"name": "str2", "type": "varchar"}] varchar $1 + $2 sql udf_db dump 2024-05-31 03:23:46 2024-05-31 03:23:46 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci -3 subab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 - $2 sql udf_db2 dump 2024-05-31 03:23:46 2024-05-31 03:23:46 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +1 addab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db dump 2024-12-31 03:03:10 2024-12-31 03:03:10 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +2 concatenate 0 [{"name": "str1", "type": "varchar"}, {"name": "str2", "type": "varchar"}] varchar $1 + $2 sql udf_db dump 2024-12-31 03:03:10 2024-12-31 03:03:10 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 subab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 - $2 sql udf_db2 dump 2024-12-31 03:03:10 2024-12-31 03:03:10 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -udf_dsp03 2024-05-31 03:23:46.287773 account sys -udf_dsp02 2024-05-31 03:23:46.138061 account sys -udf_dsp01 2024-05-31 03:23:46.078875 account sys +udf_dsp03 2024-12-31 03:03:10.602564 account sys +udf_dsp02 2024-12-31 03:03:10.421479 account sys +udf_dsp01 2024-12-31 03:03:10.325715 account sys restore account sys from snapshot udf_dsp02; select * from mo_catalog.mo_user_defined_function; function_id name owner args rettype body language db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -1 addab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db dump 2024-05-31 03:23:46 2024-05-31 03:23:46 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci -2 concatenate 0 [{"name": "str1", "type": "varchar"}, {"name": "str2", "type": "varchar"}] varchar $1 + $2 sql udf_db dump 2024-05-31 03:23:46 2024-05-31 03:23:46 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +1 addab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db dump 2024-12-31 03:03:10 2024-12-31 03:03:10 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +2 concatenate 0 [{"name": "str1", "type": "varchar"}, {"name": "str2", "type": "varchar"}] varchar $1 + $2 sql udf_db dump 2024-12-31 03:03:10 2024-12-31 03:03:10 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -udf_dsp03 2024-05-31 03:23:46.287773 account sys -udf_dsp02 2024-05-31 03:23:46.138061 account sys -udf_dsp01 2024-05-31 03:23:46.078875 account sys +udf_dsp03 2024-12-31 03:03:10.602564 account sys +udf_dsp02 2024-12-31 03:03:10.421479 account sys +udf_dsp01 2024-12-31 03:03:10.325715 account sys drop snapshot udf_dsp01; drop snapshot udf_dsp02; drop snapshot udf_dsp03; @@ -91,16 +91,16 @@ language sql as '$1 + $2'; select * from mo_catalog.mo_user_defined_function; function_id name owner args rettype body language db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 addab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db2 dump 2024-05-31 03:23:47 2024-05-31 03:23:47 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 addab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db2 dump 2024-12-31 03:03:11 2024-12-31 03:03:11 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot if exists udf_sp04; -create snapshot udf_sp04 for account sys; +create snapshot udf_sp04 for account; drop database udf_db2; select * from mo_catalog.mo_user_defined_function; function_id name owner args rettype body language db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation restore account sys from snapshot udf_sp04; select * from mo_catalog.mo_user_defined_function; function_id name owner args rettype body language db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 addab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db2 dump 2024-05-31 03:23:47 2024-05-31 03:23:47 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 addab 0 [{"name": "x", "type": "int"}, {"name": "y", "type": "int"}] int $1 + $2 sql udf_db2 dump 2024-12-31 03:03:11 2024-12-31 03:03:11 FUNCTION DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop database udf_db2; drop snapshot udf_sp04; drop database if exists procedure_test; @@ -131,17 +131,17 @@ id val 3 c select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -1 test_if_hit_elseif_first_elseif null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:48 2024-05-31 03:23:48 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci -2 test_if_hit_if null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:48 2024-05-31 03:23:48 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +1 test_if_hit_elseif_first_elseif null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-12-31 03:03:14 2024-12-31 03:03:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +2 test_if_hit_if null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-12-31 03:03:14 2024-12-31 03:03:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot if exists sp_sp05; -create snapshot sp_sp05 for account sys; +create snapshot sp_sp05 for account; drop procedure test_if_hit_elseif_first_elseif; drop procedure test_if_hit_if; restore account sys from snapshot sp_sp05; select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -1 test_if_hit_elseif_first_elseif null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:48 2024-05-31 03:23:48 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci -2 test_if_hit_if null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:48 2024-05-31 03:23:48 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +1 test_if_hit_elseif_first_elseif null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-12-31 03:03:14 2024-12-31 03:03:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +2 test_if_hit_if null {} begin DECLARE v1 INT; SET v1 = 5; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-12-31 03:03:14 2024-12-31 03:03:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci call test_if_hit_elseif_first_elseif(); id val 1 a @@ -182,16 +182,16 @@ id val 3 3.5 select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:49 2024-05-31 03:23:49 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci -4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:49 2024-05-31 03:23:49 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-12-31 03:03:15 2024-12-31 03:03:15 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-12-31 03:03:15 2024-12-31 03:03:15 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot if exists sp_sp06; -create snapshot sp_sp06 for account sys; +create snapshot sp_sp06 for account; drop table tbh1; drop table tbh2; drop procedure test_if_hit_second_elseif; select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:49 2024-05-31 03:23:49 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-12-31 03:03:15 2024-12-31 03:03:15 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci restore account sys from snapshot sp_sp06; call test_if_hit_else(); id val @@ -203,8 +203,8 @@ id val 1 a select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:49 2024-05-31 03:23:49 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci -4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:49 2024-05-31 03:23:49 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-12-31 03:03:15 2024-12-31 03:03:15 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-12-31 03:03:15 2024-12-31 03:03:15 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot sp_sp06; drop procedure test_if_hit_second_elseif; drop procedure test_if_hit_else; @@ -215,24 +215,24 @@ drop stage if exists my_ext_stage1; create stage my_ext_stage1 URL='s3://load/files/' CREDENTIALS={'AWS_KEY_ID'='1a2b3c' ,'AWS_SECRET_KEY'='4x5y6z'}; select * from mo_catalog.mo_stages; stage_id stage_name url stage_credentials stage_status created_time comment -1 my_ext_stage s3://load/files/ disabled 2024-05-31 03:23:50 -2 my_ext_stage1 s3://load/files/ *0E0591E0E0FC0806815D04D93FD4BD3F5ADB39AD disabled 2024-05-31 03:23:50 +1 my_ext_stage s3://load/files/ disabled 2024-12-31 03:03:16 +2 my_ext_stage1 s3://load/files/ AWS_KEY_ID=1a2b3c,AWS_SECRET_KEY=4x5y6z disabled 2024-12-31 03:03:16 drop snapshot if exists stage_sp01; -create snapshot stage_sp01 for account sys; +create snapshot stage_sp01 for account; alter stage my_ext_stage1 SET URL='s3://load/files2/'; select * from mo_catalog.mo_stages; stage_id stage_name url stage_credentials stage_status created_time comment -1 my_ext_stage s3://load/files/ disabled 2024-05-31 03:23:50 -2 my_ext_stage1 s3://load/files2/ *0E0591E0E0FC0806815D04D93FD4BD3F5ADB39AD disabled 2024-05-31 03:23:50 +1 my_ext_stage s3://load/files/ disabled 2024-12-31 03:03:16 +2 my_ext_stage1 s3://load/files2/ AWS_KEY_ID=1a2b3c,AWS_SECRET_KEY=4x5y6z disabled 2024-12-31 03:03:16 drop stage my_ext_stage; select * from mo_catalog.mo_stages; stage_id stage_name url stage_credentials stage_status created_time comment -2 my_ext_stage1 s3://load/files2/ *0E0591E0E0FC0806815D04D93FD4BD3F5ADB39AD disabled 2024-05-31 03:23:50 +2 my_ext_stage1 s3://load/files2/ AWS_KEY_ID=1a2b3c,AWS_SECRET_KEY=4x5y6z disabled 2024-12-31 03:03:16 restore account sys from snapshot stage_sp01; select * from mo_catalog.mo_stages; stage_id stage_name url stage_credentials stage_status created_time comment -1 my_ext_stage s3://load/files/ disabled 2024-05-31 03:23:50 -2 my_ext_stage1 s3://load/files/ *0E0591E0E0FC0806815D04D93FD4BD3F5ADB39AD disabled 2024-05-31 03:23:50 +1 my_ext_stage s3://load/files/ disabled 2024-12-31 03:03:16 +2 my_ext_stage1 s3://load/files/ AWS_KEY_ID=1a2b3c,AWS_SECRET_KEY=4x5y6z disabled 2024-12-31 03:03:16 drop snapshot stage_sp01; drop stage my_ext_stage; drop stage my_ext_stage1; @@ -247,11 +247,11 @@ dump *832EB84CB764129D05D498ED9CA7E5CE9B8F83EB unlock PASSWORD 0 userx *832EB84CB764129D05D498ED9CA7E5CE9B8F83EB unlock PASSWORD 1 0 1 usery *899ECD04E40F745BD52A4C552BE4A818AC65FAF8 unlock PASSWORD 1 0 1 drop snapshot if exists user_sp01; -create snapshot user_sp01 for account sys; +create snapshot user_sp01 for account; drop user if exists userz; create user userz identified by '111'; drop snapshot if exists user_sp02; -create snapshot user_sp02 for account sys; +create snapshot user_sp02 for account; restore account sys from snapshot user_sp01; select user_name, authentication_string, status, login_type, creator, owner, default_role from mo_catalog.mo_user; user_name authentication_string status login_type creator owner default_role @@ -283,7 +283,7 @@ public 0 0 role1 1 0 role2 1 0 drop snapshot if exists role_sp01; -create snapshot role_sp01 for account sys; +create snapshot role_sp01 for account; drop role role1; drop role role2; restore account sys from snapshot role_sp01; @@ -318,7 +318,7 @@ test_role 14 false test_role 29 false test_role 38 false drop snapshot if exists prvis_sp01; -create snapshot prvis_sp01 for account sys; +create snapshot prvis_sp01 for account; drop role test_role; select role_name, creator, owner from mo_catalog.mo_role; role_name creator owner @@ -374,7 +374,7 @@ role_account_priv_1 12 true role_account_priv_1 13 true role_account_priv_1 30 false drop snapshot if exists grant_sp01; -create snapshot grant_sp01 for account sys; +create snapshot grant_sp01 for account; drop user user_grant_2; drop role 'role_account_priv_1'; restore account sys from snapshot grant_sp01; @@ -409,7 +409,7 @@ create user if not exists user_grant_3 identified by '123456'; drop role if exists role_account_priv_3; create role 'role_account_priv_3'; drop snapshot if exists grant_sp02; -create snapshot grant_sp02 for account sys; +create snapshot grant_sp02 for account; select user_name, authentication_string, status, login_type, creator, owner, default_role from mo_catalog.mo_user; user_name authentication_string status login_type creator owner default_role root *832EB84CB764129D05D498ED9CA7E5CE9B8F83EB unlock PASSWORD 0 0 0 @@ -499,7 +499,7 @@ r3 32 false r4 32 false r5 32 false drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; drop role r1,r2,r3,r4,r5; select role_name, creator, owner from mo_catalog.mo_role; role_name creator owner @@ -581,7 +581,7 @@ r2 31 true r1 32 true r2 32 true drop snapshot if exists sp02; -create snapshot sp02 for account sys; +create snapshot sp02 for account; drop role r1, r2; select role_name, creator, owner from mo_catalog.mo_role; role_name creator owner @@ -645,7 +645,7 @@ select role_name, privilege_id, with_grant_option from mo_catalog.mo_role_privs role_name privilege_id with_grant_option r5 6 false drop snapshot if exists sp03; -create snapshot sp03 for account sys; +create snapshot sp03 for account; drop role r5; drop user user01, user02, user03; select user_name,role_name,obj_type,privilege_name,privilege_level from mo_catalog.mo_user_grant,mo_catalog.mo_user,mo_catalog.mo_role_privs where mo_user_grant.user_id=mo_user.user_id and mo_role_privs.role_id=mo_user_grant.role_id and role_name in ('r5'); @@ -712,7 +712,7 @@ operation_role_id operation_user_id 0 1 0 1 drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; revoke role_r2 from role_r3; revoke role_r1 from role_r2; select operation_role_id,operation_user_id from mo_catalog.mo_role_grant; diff --git a/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.sql b/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.sql index 87510f40d1c20..793edaeb812cd 100644 --- a/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.sql +++ b/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.sql @@ -13,7 +13,7 @@ select addab(10, 5); select name, db from mo_catalog.mo_user_defined_function; drop snapshot if exists udf_dsp01; -create snapshot udf_dsp01 for account sys; +create snapshot udf_dsp01 for account; -- @ignore:0,1 show snapshots; @@ -24,7 +24,7 @@ language sql as select concatenate('Hello, ', 'World!'); drop snapshot if exists udf_dsp02; -create snapshot udf_dsp02 for account sys; +create snapshot udf_dsp02 for account; -- @ignore:0,1 show snapshots; @@ -39,7 +39,7 @@ select subab(10, 5); select name, db from mo_catalog.mo_user_defined_function; drop snapshot if exists udf_dsp03; -create snapshot udf_dsp03 for account sys; +create snapshot udf_dsp03 for account; -- @ignore:0,1 show snapshots; @@ -80,7 +80,7 @@ language sql as select * from mo_catalog.mo_user_defined_function; drop snapshot if exists udf_sp04; -create snapshot udf_sp04 for account sys; +create snapshot udf_sp04 for account; drop database udf_db2; select * from mo_catalog.mo_user_defined_function; @@ -121,7 +121,7 @@ call test_if_hit_if(); select * from mo_catalog.mo_stored_procedure; drop snapshot if exists sp_sp05; -create snapshot sp_sp05 for account sys; +create snapshot sp_sp05 for account; drop procedure test_if_hit_elseif_first_elseif; drop procedure test_if_hit_if; @@ -167,7 +167,7 @@ call test_if_hit_else(); select * from mo_catalog.mo_stored_procedure; drop snapshot if exists sp_sp06; -create snapshot sp_sp06 for account sys; +create snapshot sp_sp06 for account; drop table tbh1; drop table tbh2; @@ -199,7 +199,7 @@ create stage my_ext_stage1 URL='s3://load/files/' CREDENTIALS={'AWS_KEY_ID'='1a2 select * from mo_catalog.mo_stages; drop snapshot if exists stage_sp01; -create snapshot stage_sp01 for account sys; +create snapshot stage_sp01 for account; alter stage my_ext_stage1 SET URL='s3://load/files2/'; -- @ignore:0,2,3,5 @@ -226,13 +226,13 @@ create user usery identified by '222'; select user_name, authentication_string, status, login_type, creator, owner, default_role from mo_catalog.mo_user; drop snapshot if exists user_sp01; -create snapshot user_sp01 for account sys; +create snapshot user_sp01 for account; drop user if exists userz; create user userz identified by '111'; drop snapshot if exists user_sp02; -create snapshot user_sp02 for account sys; +create snapshot user_sp02 for account; restore account sys from snapshot user_sp01; select user_name, authentication_string, status, login_type, creator, owner, default_role from mo_catalog.mo_user; @@ -255,7 +255,7 @@ create role role1; create role role2; select role_name, creator, owner from mo_catalog.mo_role; drop snapshot if exists role_sp01; -create snapshot role_sp01 for account sys; +create snapshot role_sp01 for account; drop role role1; drop role role2; @@ -285,7 +285,7 @@ select role_name, creator, owner from mo_catalog.mo_role; select role_name, privilege_id, with_grant_option from mo_catalog.mo_role_privs where role_name='test_role'; drop snapshot if exists prvis_sp01; -create snapshot prvis_sp01 for account sys; +create snapshot prvis_sp01 for account; drop role test_role; select role_name, creator, owner from mo_catalog.mo_role; @@ -317,7 +317,7 @@ select role_name, creator, owner from mo_catalog.mo_role; select role_name, privilege_id, with_grant_option from mo_catalog.mo_role_privs where role_name='role_account_priv_1'; drop snapshot if exists grant_sp01; -create snapshot grant_sp01 for account sys; +create snapshot grant_sp01 for account; drop user user_grant_2; drop role 'role_account_priv_1'; @@ -341,7 +341,7 @@ create user if not exists user_grant_3 identified by '123456'; drop role if exists role_account_priv_3; create role 'role_account_priv_3'; drop snapshot if exists grant_sp02; -create snapshot grant_sp02 for account sys; +create snapshot grant_sp02 for account; select user_name, authentication_string, status, login_type, creator, owner, default_role from mo_catalog.mo_user; select role_name, creator, owner from mo_catalog.mo_role; @@ -377,7 +377,7 @@ select role_name, creator, owner from mo_catalog.mo_role; select role_name, privilege_id, with_grant_option from mo_catalog.mo_role_privs where role_name in ('r1','r2','r3','r4','r5'); drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; drop role r1,r2,r3,r4,r5; select role_name, creator, owner from mo_catalog.mo_role; @@ -404,7 +404,7 @@ select role_name, creator, owner from mo_catalog.mo_role; select role_name, privilege_id, with_grant_option from mo_catalog.mo_role_privs where role_name in ('r1', 'r2'); drop snapshot if exists sp02; -create snapshot sp02 for account sys; +create snapshot sp02 for account; drop role r1, r2; select role_name, creator, owner from mo_catalog.mo_role; @@ -438,7 +438,7 @@ select role_name, creator, owner from mo_catalog.mo_role; select role_name, privilege_id, with_grant_option from mo_catalog.mo_role_privs where role_name in ('r5'); drop snapshot if exists sp03; -create snapshot sp03 for account sys; +create snapshot sp03 for account; drop role r5; drop user user01, user02, user03; @@ -475,7 +475,7 @@ select role_name, privilege_id, with_grant_option from mo_catalog.mo_role_privs select operation_role_id,operation_user_id from mo_catalog.mo_role_grant; drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; revoke role_r2 from role_r3; revoke role_r1 from role_r2; diff --git a/test/distributed/cases/snapshot/sys_restore_to_sys_account.result b/test/distributed/cases/snapshot/sys_restore_to_sys_account.result index 0c32e889ea95c..91696944d9844 100644 --- a/test/distributed/cases/snapshot/sys_restore_to_sys_account.result +++ b/test/distributed/cases/snapshot/sys_restore_to_sys_account.result @@ -3,7 +3,7 @@ use mo_catalog; create cluster table clu01(col1 int, col2 decimal); insert into clu01 values(1,2,0); drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; insert into clu01 values(2,3,0); restore account sys from snapshot sp01; select * from clu01; @@ -16,7 +16,7 @@ use test; create table clu01(col1 int, col2 decimal); insert into clu01 values(1,2); drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; insert into clu01 values(2,3); restore account sys from snapshot sp01; select * from clu01; @@ -42,7 +42,7 @@ select count(*) from rs01; count(*) 6 drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; select count(*) from rs01 {snapshot = 'sp01'}; count(*) 6 @@ -63,7 +63,7 @@ count(*) 1 select * from mo_catalog.mo_database{snapshot = 'sp01'} where datname = 'test01'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -273476 test01 def create database test01 0 1 2024-12-16 17:56:22 0 +274235 test01 def create database test01 0 1 2024-12-31 10:40:41 0 select attname from mo_catalog.mo_columns{snapshot = 'sp01'} where att_database = 'test01'; attname col1 @@ -88,7 +88,7 @@ count(*) 1 select * from mo_catalog.mo_database{snapshot = 'sp01'} where datname = 'test01'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -273476 test01 def create database test01 0 1 2024-12-16 17:56:22 0 +274235 test01 def create database test01 0 1 2024-12-31 10:40:41 0 select attname from mo_catalog.mo_columns{snapshot = 'sp01'} where att_database = 'test01'; attname col1 @@ -136,16 +136,16 @@ insert into rs03 values (2, -12.1, 34738, null); insert into rs03 values (1, 90.2314, null, 4); insert into rs03 values (1, 43425.4325, -7483.432, 2); drop snapshot if exists sp02; -create snapshot sp02 for account sys; +create snapshot sp02 for account; select count(*) from mo_catalog.mo_tables{snapshot = 'sp02'} where reldatabase = 'test02'; count(*) 2 select * from mo_catalog.mo_database{snapshot = 'sp02'} where datname = 'test02'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -273504 test02 def create database test02 0 1 2024-12-16 17:56:24 0 +274263 test02 def create database test02 0 1 2024-12-31 10:40:42 0 select * from mo_catalog.mo_database{snapshot = 'sp02'} where datname = 'mo_catalog'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -1 mo_catalog mo_catalog 0 0 2024-12-16 17:53:47 0 +1 mo_catalog mo_catalog 0 0 2024-12-30 10:46:53 0 select attname from mo_catalog.mo_columns{snapshot = 'sp02'} where att_database = 'test02'; attname col1 @@ -270,10 +270,10 @@ internal error: find 0 snapshot records by name(sp04), expect only 1 select attname from mo_catalog.mo_columns{snapshot = 'sp04'} where att_database = 'test03'; internal error: find 0 snapshot records by name(sp04), expect only 1 drop snapshot if exists sp04; -create snapshot sp04 for account sys; +create snapshot sp04 for account; show snapshots where snapshot_name = 'sp04'; snapshot_name timestamp snapshot_level account_name database_name table_name -sp04 2024-12-16 09:56:26.988168 account sys +sp04 2024-12-31 02:40:44.697756 account sys select * from aff01{snapshot = 'sp04'}; empno ename job mgr hiredate sal comm deptno 7369 SMITH CLERK 7902 1980-12-17 800.00 null 20 @@ -397,7 +397,7 @@ select count(*) from t3; count(*) 3 drop snapshot if exists snap01; -create snapshot snap01 for account sys; +create snapshot snap01 for account; select count(*) from test01.t1 {snapshot = 'snap01'}; count(*) 3 @@ -477,7 +477,7 @@ drop database test02; drop database test03; drop snapshot snap01; drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; create database db01; restore account sys FROM snapshot sp05; show databases; @@ -502,15 +502,15 @@ drop table if exists table02; create table table02 (col1 int unique key, col2 varchar(20)); insert into table02 (col1, col2) values (133, 'database'); drop snapshot if exists sp07; -create snapshot sp07 for account sys; +create snapshot sp07 for account; drop table table01; insert into table02 values(134, 'database'); drop snapshot if exists sp08; -create snapshot sp08 for account sys; +create snapshot sp08 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sp08 2024-12-16 09:56:38.090964 account sys -sp07 2024-12-16 09:56:37.988555 account sys +sp08 2024-12-31 02:41:02.437582 account sys +sp07 2024-12-31 02:41:02.312412 account sys restore account sys from snapshot sp08; select * from table02; col1 col2 @@ -549,15 +549,15 @@ drop table if exists table02; create table table02 (col1 int unique key, col2 varchar(20)); insert into table02 (col1, col2) values (133, 'database'); drop snapshot if exists sp09; -create snapshot sp09 for account sys; +create snapshot sp09 for account; alter table table01 drop primary key; insert into table02 values(134, 'database'); drop snapshot if exists sp10; -create snapshot sp10 for account sys; +create snapshot sp10 for account; show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sp10 2024-12-16 09:56:40.775367 account sys -sp09 2024-12-16 09:56:40.666321 account sys +sp10 2024-12-31 02:41:04.864659 account sys +sp09 2024-12-31 02:41:04.741147 account sys restore account sys from snapshot sp09; select * from table02; col1 col2 @@ -617,7 +617,7 @@ show create table tm2; Table Create Table tm2 CREATE TABLE `tm2` (\n `a` int NOT NULL AUTO_INCREMENT,\n `b` int DEFAULT NULL,\n `c` int DEFAULT NULL,\n PRIMARY KEY (`a`)\n) drop snapshot if exists sp11; -create snapshot sp11 for account sys; +create snapshot sp11 for account; insert into ti1 values (1,1,1), (2,2,2); insert into ti2 values (1,1,1), (2,2,2); select * from ti1; @@ -633,7 +633,7 @@ a b c select * from tm2; a b c drop snapshot if exists sp12; -create snapshot sp12 for account sys; +create snapshot sp12 for account; insert into tm1 values (1,1,1), (2,2,2); insert into tm2 values (1,1,1), (2,2,2); select * from ti1 {snapshot = 'sp12'}; @@ -653,11 +653,11 @@ a b c 1 1 1 2 2 2 drop snapshot if exists sp13; -create snapshot sp13 for account sys; +create snapshot sp13 for account; alter table ti1 add constraint fi1 foreign key (b) references ti2(a); alter table tm1 add constraint fm1 foreign key (b) references tm2(a); drop snapshot if exists sp14; -create snapshot sp14 for account sys; +create snapshot sp14 for account; show create table ti1 {snapshot = 'sp14'}; Table Create Table ti1 CREATE TABLE `ti1` (\n `a` int NOT NULL,\n `b` int DEFAULT NULL,\n `c` int DEFAULT NULL,\n CONSTRAINT `fi1` FOREIGN KEY (`b`) REFERENCES `ti2` (`a`) ON DELETE RESTRICT ON UPDATE RESTRICT\n) @@ -675,7 +675,7 @@ alter table tm1 drop foreign key fm1; truncate ti2; truncate tm2; drop snapshot if exists sp15; -create snapshot sp15 for account sys; +create snapshot sp15 for account; show create table ti1 {snapshot = 'sp14'}; Table Create Table ti1 CREATE TABLE `ti1` (\n `a` int NOT NULL,\n `b` int DEFAULT NULL,\n `c` int DEFAULT NULL,\n CONSTRAINT `fi1` FOREIGN KEY (`b`) REFERENCES `ti2` (`a`) ON DELETE RESTRICT ON UPDATE RESTRICT\n) @@ -809,7 +809,7 @@ col1 1 2 drop snapshot if exists sp13; -create snapshot sp13 for account sys; +create snapshot sp13 for account; insert into db04.table01 values (200); insert into db05.table01 values (400); restore account sys database db04 from snapshot sp13; @@ -874,7 +874,7 @@ show create table db07.table02; Table Create Table table02 CREATE TABLE `table02` (\n `col1` int DEFAULT NULL,\n `col2` enum('a','b','c') DEFAULT NULL\n) drop snapshot if exists sp14; -create snapshot sp14 for account sys; +create snapshot sp14 for account; use db07; drop table table01; alter table table02 add column newC int first; @@ -886,7 +886,7 @@ Table Create Table table02 CREATE TABLE `table02` (\n `col1` int DEFAULT NULL,\n `col2` enum('a','b','c') DEFAULT NULL\n) drop database db08; drop snapshot if exists sp15; -create snapshot sp15 for account sys; +create snapshot sp15 for account; restore account sys database db07 from snapshot sp15; restore account sys database db08 from snapshot sp15; Unknown database db08 @@ -914,8 +914,8 @@ select * from table01; SQL parser error: table "table01" does not exist show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sp15 2024-12-16 09:56:59.876794 account sys -sp14 2024-12-16 09:56:58.785736 account sys +sp15 2024-12-31 02:41:21.678227 account sys +sp14 2024-12-31 02:41:21.402114 account sys drop snapshot sp14; drop snapshot sp15; drop database db07; @@ -927,7 +927,7 @@ use db08; create table db08 (col1 int); insert into db08 values(1), (100), (20000); drop snapshot if exists sp15; -create snapshot sp15 for account sys; +create snapshot sp15 for account; insert into db08 (col1) values (3000); restore account sys database db from snapshot sp15; Unknown database db @@ -979,7 +979,7 @@ partition p04 values less than (400001) insert into index03 values (9001,'1980-12-17', 'SMITH', 'CLERK', 'F', '2008-12-17'), (9002,'1981-02-20', 'ALLEN', 'SALESMAN', 'F', '2008-02-20'); drop snapshot if exists sp16; -create snapshot sp16 for account sys; +create snapshot sp16 for account; use db09; delete from index02 where col1 = 'a'; select * from index02; @@ -1017,7 +1017,7 @@ emp_no birth_date first_name last_name gender hire_date 9002 1981-02-20 ALLEN SALESMAN F 2008-02-20 show snapshots; SNAPSHOT_NAME TIMESTAMP SNAPSHOT_LEVEL ACCOUNT_NAME DATABASE_NAME TABLE_NAME -sp16 2024-12-16 09:57:01.043206 account sys +sp16 2024-12-31 02:41:22.97168 account sys drop database db09; drop database db10; drop snapshot sp16; @@ -1033,12 +1033,12 @@ col1 col2 1 2378 234 -3923 drop snapshot if exists sp17; -create snapshot sp17 for account sys; +create snapshot sp17 for account; alter table pri01 add constraint primary key(col1); insert into pri01 values (23423, 32432543.3242); insert into pri01 values (234222, -3923.2342342); drop snapshot if exists sp18; -create snapshot sp18 for account sys; +create snapshot sp18 for account; restore account sys database db11 table pri01 from snapshot sp18; show create table pri01; Table Create Table @@ -1093,7 +1093,7 @@ count(*) 1 select * from mo_catalog.mo_database{snapshot = 'sp18'} where datname = 'db11'; dat_id datname dat_catalog_name dat_createsql owner creator created_time account_id dat_type -273846 db11 def create database db11 0 1 2024-12-16 17:57:01 0 +274616 db11 def create database db11 0 1 2024-12-31 10:41:23 0 select attname from mo_catalog.mo_columns{snapshot = 'sp18'} where att_database = 'db11'; attname col1 @@ -1119,7 +1119,7 @@ comment_text text, foreign key (post_id) references posts(post_id) ); drop snapshot if exists sp_fulltext; -create snapshot sp_fulltext for account sys; +create snapshot sp_fulltext for account; alter table posts add fulltext(content); insert into posts (title, content) values ('MO全文索引入门', 'MO全文索引是一种强大的工具,可以帮助你快速检索数据库中的文本数据。'), diff --git a/test/distributed/cases/snapshot/sys_restore_to_sys_account.sql b/test/distributed/cases/snapshot/sys_restore_to_sys_account.sql index 7f18c6c1f14eb..eedcb2c713e67 100644 --- a/test/distributed/cases/snapshot/sys_restore_to_sys_account.sql +++ b/test/distributed/cases/snapshot/sys_restore_to_sys_account.sql @@ -4,7 +4,7 @@ create cluster table clu01(col1 int, col2 decimal); insert into clu01 values(1,2,0); drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; insert into clu01 values(2,3,0); restore account sys from snapshot sp01; @@ -22,7 +22,7 @@ create table clu01(col1 int, col2 decimal); insert into clu01 values(1,2); drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; insert into clu01 values(2,3); restore account sys from snapshot sp01; @@ -51,7 +51,7 @@ insert into rs01 values (2, -1, null); select count(*) from rs01; drop snapshot if exists sp01; -create snapshot sp01 for account sys; +create snapshot sp01 for account; select count(*) from rs01 {snapshot = 'sp01'}; insert into rs01 values (2, -1, null); insert into rs01 values (1, -321.321, null); @@ -102,7 +102,7 @@ insert into rs03 values (2, -12.1, 34738, null); insert into rs03 values (1, 90.2314, null, 4); insert into rs03 values (1, 43425.4325, -7483.432, 2); drop snapshot if exists sp02; -create snapshot sp02 for account sys; +create snapshot sp02 for account; select count(*) from mo_catalog.mo_tables{snapshot = 'sp02'} where reldatabase = 'test02'; -- @ignore:0,5,6,7 select * from mo_catalog.mo_database{snapshot = 'sp02'} where datname = 'test02'; @@ -201,7 +201,7 @@ select * from mo_catalog.mo_database{snapshot = 'sp04'} where datname = 'test03' select attname from mo_catalog.mo_columns{snapshot = 'sp04'} where att_database = 'test03'; drop snapshot if exists sp04; -create snapshot sp04 for account sys; +create snapshot sp04 for account; -- @ignore:1 show snapshots where snapshot_name = 'sp04'; @@ -271,7 +271,7 @@ select count(*) from t2; select count(*) from t3; drop snapshot if exists snap01; -create snapshot snap01 for account sys; +create snapshot snap01 for account; select count(*) from test01.t1 {snapshot = 'snap01'}; select count(*) from test02.t2 {snapshot = 'snap01'}; select count(*) from test03.t3 {snapshot = 'snap01'}; @@ -305,7 +305,7 @@ drop snapshot snap01; -- restore null drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; create database db01; restore account sys FROM snapshot sp05; show databases; @@ -327,13 +327,13 @@ create table table02 (col1 int unique key, col2 varchar(20)); insert into table02 (col1, col2) values (133, 'database'); drop snapshot if exists sp07; -create snapshot sp07 for account sys; +create snapshot sp07 for account; drop table table01; insert into table02 values(134, 'database'); drop snapshot if exists sp08; -create snapshot sp08 for account sys; +create snapshot sp08 for account; -- @ignore:1 show snapshots; restore account sys from snapshot sp08; @@ -368,13 +368,13 @@ create table table02 (col1 int unique key, col2 varchar(20)); insert into table02 (col1, col2) values (133, 'database'); drop snapshot if exists sp09; -create snapshot sp09 for account sys; +create snapshot sp09 for account; alter table table01 drop primary key; insert into table02 values(134, 'database'); drop snapshot if exists sp10; -create snapshot sp10 for account sys; +create snapshot sp10 for account; -- @ignore:1 show snapshots; restore account sys from snapshot sp09; @@ -415,7 +415,7 @@ show create table tm1; show create table ti2; show create table tm2; drop snapshot if exists sp11; -create snapshot sp11 for account sys; +create snapshot sp11 for account; insert into ti1 values (1,1,1), (2,2,2); insert into ti2 values (1,1,1), (2,2,2); @@ -424,7 +424,7 @@ select * from tm1; select * from ti2; select * from tm2; drop snapshot if exists sp12; -create snapshot sp12 for account sys; +create snapshot sp12 for account; insert into tm1 values (1,1,1), (2,2,2); insert into tm2 values (1,1,1), (2,2,2); @@ -433,12 +433,12 @@ select * from tm1; select * from ti2 {snapshot = 'sp12'}; select * from tm2; drop snapshot if exists sp13; -create snapshot sp13 for account sys; +create snapshot sp13 for account; alter table ti1 add constraint fi1 foreign key (b) references ti2(a); alter table tm1 add constraint fm1 foreign key (b) references tm2(a); drop snapshot if exists sp14; -create snapshot sp14 for account sys; +create snapshot sp14 for account; show create table ti1 {snapshot = 'sp14'}; show create table tm1 {snapshot = 'sp13'}; @@ -450,7 +450,7 @@ alter table tm1 drop foreign key fm1; truncate ti2; truncate tm2; drop snapshot if exists sp15; -create snapshot sp15 for account sys; +create snapshot sp15 for account; show create table ti1 {snapshot = 'sp14'}; show create table tm1 {snapshot = 'sp15'}; @@ -518,7 +518,7 @@ select * from db05.table01; select * from db06.table01; drop snapshot if exists sp13; -create snapshot sp13 for account sys; +create snapshot sp13 for account; insert into db04.table01 values (200); insert into db05.table01 values (400); @@ -561,7 +561,7 @@ insert into index01 values (2); show create table db07.table02; drop snapshot if exists sp14; -create snapshot sp14 for account sys; +create snapshot sp14 for account; use db07; drop table table01; @@ -571,7 +571,7 @@ show create table table02 {snapshot = 'sp14'}; drop database db08; drop snapshot if exists sp15; -create snapshot sp15 for account sys; +create snapshot sp15 for account; restore account sys database db07 from snapshot sp15; restore account sys database db08 from snapshot sp15; @@ -600,7 +600,7 @@ use db08; create table db08 (col1 int); insert into db08 values(1), (100), (20000); drop snapshot if exists sp15; -create snapshot sp15 for account sys; +create snapshot sp15 for account; insert into db08 (col1) values (3000); restore account sys database db from snapshot sp15; @@ -657,7 +657,7 @@ insert into index03 values (9001,'1980-12-17', 'SMITH', 'CLERK', 'F', '2008-12-1 (9002,'1981-02-20', 'ALLEN', 'SALESMAN', 'F', '2008-02-20'); drop snapshot if exists sp16; -create snapshot sp16 for account sys; +create snapshot sp16 for account; use db09; delete from index02 where col1 = 'a'; @@ -699,14 +699,14 @@ insert into pri01 values (234, -3923.2342342); select * from pri01; drop snapshot if exists sp17; -create snapshot sp17 for account sys; +create snapshot sp17 for account; alter table pri01 add constraint primary key(col1); insert into pri01 values (23423, 32432543.3242); insert into pri01 values (234222, -3923.2342342); drop snapshot if exists sp18; -create snapshot sp18 for account sys; +create snapshot sp18 for account; restore account sys database db11 table pri01 from snapshot sp18; show create table pri01; @@ -754,7 +754,7 @@ create table comments ( foreign key (post_id) references posts(post_id) ); drop snapshot if exists sp_fulltext; -create snapshot sp_fulltext for account sys; +create snapshot sp_fulltext for account; alter table posts add fulltext(content); insert into posts (title, content) values ('MO全文索引入门', 'MO全文索引是一种强大的工具,可以帮助你快速检索数据库中的文本数据。'), diff --git a/test/distributed/cases/snapshot/sys_restore_view_to_sys_account.result b/test/distributed/cases/snapshot/sys_restore_view_to_sys_account.result index 77d26311c7c26..dbb2cf345d46a 100644 --- a/test/distributed/cases/snapshot/sys_restore_view_to_sys_account.result +++ b/test/distributed/cases/snapshot/sys_restore_view_to_sys_account.result @@ -39,7 +39,7 @@ col1 col2 1 -321 2 -1 drop snapshot if exists sp100; -create snapshot sp100 for account sys; +create snapshot sp100 for account; delete from table01 where col1 = 1; select * from v01; col1 col2 col3 @@ -182,7 +182,7 @@ avg(sal) 1366.66666667 2758.33333333 drop snapshot if exists sp02; -create snapshot sp02 for account sys; +create snapshot sp02 for account; drop view v04; select * from v04; SQL parser error: table "v04" does not exist @@ -267,7 +267,7 @@ Alice Johnson Marketing Mark Patterson Sales David Finley Finance drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; drop table Employees; select * from EmployeeDepartmentView; SQL parser error: table "employees" does not exist @@ -365,7 +365,7 @@ employee_id first_name last_name department_name 2 Jane Smith Engineering 3 Bob Johnson HR drop snapshot if exists sp04; -create snapshot sp04 for account sys; +create snapshot sp04 for account; select * from employee_view {snapshot = 'sp04'}; employee_id first_name last_name department_id 1 John Doe 1 @@ -458,7 +458,7 @@ emp_no birth_date first_name last_name gender hire_date 9001 1980-12-17 SMITH CLERK F 2008-12-17 9002 1981-02-20 ALLEN SALESMAN F 2008-02-20 drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; insert into partition01 values (9003,'1999-02-20', 'BOB', 'DOCTOR', 'F', '2009-02-20'); select * from view01; emp_no birth_date first_name last_name gender hire_date @@ -470,7 +470,7 @@ emp_no birth_date first_name last_name gender hire_date 9001 1980-12-17 SMITH CLERK F 2008-12-17 9002 1981-02-20 ALLEN SALESMAN F 2008-02-20 drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; restore account sys from snapshot sp05; select * from partition01; emp_no birth_date first_name last_name gender hire_date @@ -553,7 +553,7 @@ employeeid employeename salary salaryrank 1 Alice 70000.00 4 4 David 65000.00 5 drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; drop database test01; select * from test01.EmployeeSalaryRanking; Unknown database test01 @@ -640,7 +640,7 @@ Alice Mathematics Mr. Smith 2024-01-10 Bob Physics Dr. Johnson 2024-01-15 Charlie Mathematics Mr. Smith 2024-02-01 drop snapshot if exists sp07; -create snapshot sp07 for account sys; +create snapshot sp07 for account; drop table Enrollments; drop table students; select * from StudentCoursesView; @@ -727,7 +727,7 @@ select department, avg(salary) as avg_salary from employees group by department; drop snapshot if exists sp10; -create snapshot sp10 for account sys; +create snapshot sp10 for account; drop database test02; restore account sys from snapshot sp10; drop view if exists employees_view; @@ -792,7 +792,7 @@ employee_id first_name last_name department_name 2 Jane Smith Engineering 3 Bob Johnson HR drop snapshot if exists sp100; -create snapshot sp100 for account sys; +create snapshot sp100 for account; drop database test04; select * from test04.employee_view; Unknown database test04 @@ -817,7 +817,7 @@ employee_id first_name last_name department_name 2 Jane Smith Engineering 3 Bob Johnson HR drop snapshot if exists sp101; -create snapshot sp101 for account sys; +create snapshot sp101 for account; drop database test03; drop database test05; select * from test04.employee_view; diff --git a/test/distributed/cases/snapshot/sys_restore_view_to_sys_account.sql b/test/distributed/cases/snapshot/sys_restore_view_to_sys_account.sql index 14a7c5f7a36b7..638013b90be8b 100644 --- a/test/distributed/cases/snapshot/sys_restore_view_to_sys_account.sql +++ b/test/distributed/cases/snapshot/sys_restore_view_to_sys_account.sql @@ -23,7 +23,7 @@ show create view v02; select * from v02; drop snapshot if exists sp100; -create snapshot sp100 for account sys; +create snapshot sp100 for account; delete from table01 where col1 = 1; select * from v01; @@ -100,7 +100,7 @@ create view v05 as select * from v04; select * from v05; drop snapshot if exists sp02; -create snapshot sp02 for account sys; +create snapshot sp02 for account; drop view v04; select * from v04; @@ -167,7 +167,7 @@ from select * from EmployeeDepartmentView; drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; drop table Employees; select * from EmployeeDepartmentView; @@ -234,7 +234,7 @@ select * from department_view; select * from employee_with_department_view; drop snapshot if exists sp04; -create snapshot sp04 for account sys; +create snapshot sp04 for account; select * from employee_view {snapshot = 'sp04'}; select * from department_view {snapshot = 'sp04'}; @@ -298,14 +298,14 @@ create view view01 as select * from partition01; select * from view01; drop snapshot if exists sp05; -create snapshot sp05 for account sys; +create snapshot sp05 for account; insert into partition01 values (9003,'1999-02-20', 'BOB', 'DOCTOR', 'F', '2009-02-20'); select * from view01; select * from view01{snapshot = 'sp05'}; drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; restore account sys from snapshot sp05; @@ -361,7 +361,7 @@ from select * from EmployeeSalaryRanking; drop snapshot if exists sp06; -create snapshot sp06 for account sys; +create snapshot sp06 for account; drop database test01; select * from test01.EmployeeSalaryRanking; @@ -436,7 +436,7 @@ order by select * from StudentCoursesView; drop snapshot if exists sp07; -create snapshot sp07 for account sys; +create snapshot sp07 for account; drop table Enrollments; drop table students; @@ -497,7 +497,7 @@ from employees group by department; drop snapshot if exists sp10; -create snapshot sp10 for account sys; +create snapshot sp10 for account; drop database test02; @@ -567,7 +567,7 @@ from test04.employee_view e join test03.department_view d on e.department_id = d select * from employee_with_department_view; drop snapshot if exists sp100; -create snapshot sp100 for account sys; +create snapshot sp100 for account; drop database test04; @@ -579,7 +579,7 @@ select * from test03.department_view {snapshot = 'sp100'}; select * from test05.employee_with_department_view {snapshot = 'sp100'}; drop snapshot if exists sp101; -create snapshot sp101 for account sys; +create snapshot sp101 for account; drop database test03; drop database test05;