1031 lines
31 KiB
Go
1031 lines
31 KiB
Go
// Copyright 2023 PingCAP, Inc.
|
|
//
|
|
// 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 timer_test
|
|
|
|
import (
|
|
"context"
|
|
"fmt"
|
|
"strings"
|
|
"sync/atomic"
|
|
"testing"
|
|
"time"
|
|
|
|
"github.com/google/uuid"
|
|
"github.com/pingcap/errors"
|
|
"github.com/pingcap/tidb/pkg/kv"
|
|
"github.com/pingcap/tidb/pkg/meta/metadef"
|
|
"github.com/pingcap/tidb/pkg/session/syssession"
|
|
"github.com/pingcap/tidb/pkg/testkit"
|
|
"github.com/pingcap/tidb/pkg/timer/api"
|
|
"github.com/pingcap/tidb/pkg/timer/runtime"
|
|
"github.com/pingcap/tidb/pkg/timer/tablestore"
|
|
"github.com/pingcap/tidb/pkg/util/timeutil"
|
|
"github.com/stretchr/testify/require"
|
|
"github.com/tikv/client-go/v2/util"
|
|
"go.etcd.io/etcd/tests/v3/integration"
|
|
)
|
|
|
|
type mockSessionPool struct {
|
|
t *testing.T
|
|
syssession.Pool
|
|
pool syssession.Pool
|
|
inuse atomic.Bool
|
|
}
|
|
|
|
func (p *mockSessionPool) WithSession(fn func(*syssession.Session) error) error {
|
|
return p.pool.WithSession(func(s *syssession.Session) error {
|
|
require.True(p.t, p.inuse.CompareAndSwap(false, true))
|
|
defer func() {
|
|
require.True(p.t, p.inuse.CompareAndSwap(true, false))
|
|
}()
|
|
return fn(s)
|
|
})
|
|
}
|
|
|
|
func TestMemTimerStore(t *testing.T) {
|
|
timeutil.SetSystemTZ("Asia/Shanghai")
|
|
store := api.NewMemoryTimerStore()
|
|
defer store.Close()
|
|
runTimerStoreTest(t, store)
|
|
|
|
store = api.NewMemoryTimerStore()
|
|
defer store.Close()
|
|
runTimerStoreWatchTest(t, store)
|
|
}
|
|
|
|
// createTimerTableSQL returns a SQL to create timer table
|
|
func createTimerTableSQL(dbName, tableName string) string {
|
|
return strings.Replace(metadef.CreateTiDBTimersTable, "mysql.tidb_timers", fmt.Sprintf("`%s`.`%s`", dbName, tableName), 1)
|
|
}
|
|
|
|
func TestTableTimerStore(t *testing.T) {
|
|
timeutil.SetSystemTZ("Asia/Shanghai")
|
|
store, do := testkit.CreateMockStoreAndDomain(t)
|
|
tk := testkit.NewTestKit(t, store)
|
|
dbName := "test"
|
|
tblName := "timerstore"
|
|
tk.MustExec("use test")
|
|
tk.MustExec(createTimerTableSQL(dbName, tblName))
|
|
|
|
pool := &mockSessionPool{t: t, pool: do.AdvancedSysSessionPool()}
|
|
// test CURD
|
|
timerStore := tablestore.NewTableTimerStore(1, pool, dbName, tblName, nil)
|
|
defer timerStore.Close()
|
|
runTimerStoreTest(t, timerStore)
|
|
|
|
// test cluster time zone
|
|
runClusterTimeZoneTest(t, timerStore, func(tz string) {
|
|
require.NoError(t, pool.WithSession(func(s *syssession.Session) error {
|
|
ctx := util.WithInternalSourceType(context.TODO(), kv.InternalTimer)
|
|
rs, err := s.ExecuteInternal(ctx, "SET @@global.time_zone = %?", tz)
|
|
require.NoError(t, err)
|
|
require.Nil(t, rs)
|
|
return nil
|
|
}))
|
|
})
|
|
|
|
// test notifications
|
|
integration.BeforeTestExternal(t)
|
|
testEtcdCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
|
defer testEtcdCluster.Terminate(t)
|
|
|
|
cli := testEtcdCluster.RandClient()
|
|
tk.MustExec("drop table " + tblName)
|
|
tk.MustExec(createTimerTableSQL(dbName, tblName))
|
|
timerStore = tablestore.NewTableTimerStore(1, pool, dbName, tblName, cli)
|
|
defer timerStore.Close()
|
|
runTimerStoreWatchTest(t, timerStore)
|
|
|
|
// check pool
|
|
require.False(t, pool.inuse.Load())
|
|
}
|
|
|
|
func runClusterTimeZoneTest(t *testing.T, store *api.TimerStore, setClusterTZ func(string)) {
|
|
timerID, err := store.Create(context.Background(), &api.TimerRecord{
|
|
TimerSpec: api.TimerSpec{
|
|
Namespace: "n1",
|
|
Key: "/path/to/testtz",
|
|
TimeZone: "",
|
|
SchedPolicyType: api.SchedEventCron,
|
|
SchedPolicyExpr: "* 1 * * *",
|
|
},
|
|
})
|
|
require.NoError(t, err)
|
|
timer, err := store.GetByID(context.Background(), timerID)
|
|
require.NoError(t, err)
|
|
require.Equal(t, "", timer.TimeZone)
|
|
require.Equal(t, timeutil.SystemLocation(), timer.Location)
|
|
|
|
setClusterTZ("UTC")
|
|
timer, err = store.GetByID(context.Background(), timerID)
|
|
require.NoError(t, err)
|
|
require.Equal(t, "", timer.TimeZone)
|
|
require.Equal(t, time.UTC, timer.Location)
|
|
}
|
|
|
|
func runTimerStoreTest(t *testing.T, store *api.TimerStore) {
|
|
ctx := context.Background()
|
|
timer := runTimerStoreInsertAndGet(ctx, t, store)
|
|
runTimerStoreUpdate(ctx, t, store, timer)
|
|
runTimerStoreDelete(ctx, t, store, timer)
|
|
runTimerStoreInsertAndList(ctx, t, store)
|
|
}
|
|
|
|
func runTimerStoreInsertAndGet(ctx context.Context, t *testing.T, store *api.TimerStore) *api.TimerRecord {
|
|
records, err := store.List(ctx, nil)
|
|
require.NoError(t, err)
|
|
require.Empty(t, records)
|
|
|
|
recordTpl := api.TimerRecord{
|
|
TimerSpec: api.TimerSpec{
|
|
Namespace: "n1",
|
|
Key: "/path/to/key",
|
|
TimeZone: "",
|
|
SchedPolicyType: api.SchedEventInterval,
|
|
SchedPolicyExpr: "1h",
|
|
Data: []byte("data1"),
|
|
},
|
|
}
|
|
|
|
// normal insert
|
|
record := recordTpl.Clone()
|
|
id, err := store.Create(ctx, record)
|
|
require.NoError(t, err)
|
|
require.Equal(t, recordTpl, *record)
|
|
require.NotEmpty(t, id)
|
|
recordTpl.ID = id
|
|
recordTpl.Location = timeutil.SystemLocation()
|
|
recordTpl.EventStatus = api.SchedEventIdle
|
|
|
|
// get by id
|
|
got, err := store.GetByID(ctx, id)
|
|
require.NoError(t, err)
|
|
require.NotSame(t, record, got)
|
|
record = got
|
|
require.Equal(t, recordTpl.ID, record.ID)
|
|
require.NotZero(t, record.Version)
|
|
recordTpl.Version = record.Version
|
|
require.False(t, record.CreateTime.IsZero())
|
|
recordTpl.CreateTime = record.CreateTime
|
|
require.Equal(t, recordTpl, *record)
|
|
|
|
// id not exist
|
|
_, err = store.GetByID(ctx, "noexist")
|
|
require.True(t, errors.ErrorEqual(err, api.ErrTimerNotExist))
|
|
|
|
// get by key
|
|
record, err = store.GetByKey(ctx, "n1", "/path/to/key")
|
|
require.NoError(t, err)
|
|
require.Equal(t, recordTpl, *record)
|
|
|
|
// key not exist
|
|
_, err = store.GetByKey(ctx, "n1", "noexist")
|
|
require.True(t, errors.ErrorEqual(err, api.ErrTimerNotExist))
|
|
_, err = store.GetByKey(ctx, "n2", "/path/to/ke")
|
|
require.True(t, errors.ErrorEqual(err, api.ErrTimerNotExist))
|
|
|
|
// invalid insert
|
|
invalid := &api.TimerRecord{}
|
|
_, err = store.Create(ctx, invalid)
|
|
require.EqualError(t, err, "field 'Namespace' should not be empty")
|
|
|
|
invalid.Namespace = "n1"
|
|
_, err = store.Create(ctx, invalid)
|
|
require.EqualError(t, err, "field 'Key' should not be empty")
|
|
|
|
invalid.Key = "k1"
|
|
_, err = store.Create(ctx, invalid)
|
|
require.EqualError(t, err, "field 'SchedPolicyType' should not be empty")
|
|
|
|
invalid.SchedPolicyType = api.SchedEventInterval
|
|
invalid.SchedPolicyExpr = "1x"
|
|
_, err = store.Create(ctx, invalid)
|
|
require.EqualError(t, err, "schedule event configuration is not valid: invalid schedule event expr '1x': unknown unit x")
|
|
|
|
invalid.SchedPolicyExpr = "1h"
|
|
invalid.TimeZone = "tidb"
|
|
_, err = store.Create(ctx, invalid)
|
|
require.ErrorContains(t, err, "Unknown or incorrect time zone: 'tidb'")
|
|
|
|
return &recordTpl
|
|
}
|
|
|
|
func runTimerStoreUpdate(ctx context.Context, t *testing.T, store *api.TimerStore, tpl *api.TimerRecord) {
|
|
// normal update
|
|
orgRecord, err := store.GetByID(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
require.Equal(t, "1h", tpl.SchedPolicyExpr)
|
|
eventID := uuid.NewString()
|
|
eventStart := time.Unix(1234567, 0)
|
|
watermark := time.Unix(7890123, 0)
|
|
err = store.Update(ctx, tpl.ID, &api.TimerUpdate{
|
|
Tags: api.NewOptionalVal([]string{"l1", "l2"}),
|
|
TimeZone: api.NewOptionalVal("UTC"),
|
|
SchedPolicyExpr: api.NewOptionalVal("2h"),
|
|
ManualRequest: api.NewOptionalVal(api.ManualRequest{
|
|
ManualRequestID: "req1",
|
|
ManualRequestTime: time.Unix(123, 0),
|
|
ManualTimeout: time.Minute,
|
|
ManualProcessed: true,
|
|
ManualEventID: "event1",
|
|
}),
|
|
EventStatus: api.NewOptionalVal(api.SchedEventTrigger),
|
|
EventID: api.NewOptionalVal(eventID),
|
|
EventData: api.NewOptionalVal([]byte("eventdata1")),
|
|
EventStart: api.NewOptionalVal(eventStart),
|
|
EventExtra: api.NewOptionalVal(api.EventExtra{
|
|
EventManualRequestID: "req2",
|
|
EventWatermark: time.Unix(456, 0),
|
|
}),
|
|
Watermark: api.NewOptionalVal(watermark),
|
|
SummaryData: api.NewOptionalVal([]byte("summary1")),
|
|
CheckVersion: api.NewOptionalVal(orgRecord.Version),
|
|
CheckEventID: api.NewOptionalVal(""),
|
|
})
|
|
require.NoError(t, err)
|
|
record, err := store.GetByID(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
require.NotSame(t, orgRecord, record)
|
|
require.Greater(t, record.Version, tpl.Version)
|
|
tpl.Version = record.Version
|
|
tpl.TimeZone = "UTC"
|
|
tpl.Location = time.UTC
|
|
tpl.SchedPolicyExpr = "2h"
|
|
tpl.Tags = []string{"l1", "l2"}
|
|
tpl.EventStatus = api.SchedEventTrigger
|
|
tpl.EventID = eventID
|
|
tpl.EventData = []byte("eventdata1")
|
|
require.Equal(t, eventStart.Unix(), record.EventStart.Unix())
|
|
tpl.EventStart = record.EventStart
|
|
require.Equal(t, watermark.Unix(), record.Watermark.Unix())
|
|
tpl.Watermark = record.Watermark
|
|
tpl.SummaryData = []byte("summary1")
|
|
tpl.ManualRequest = api.ManualRequest{
|
|
ManualRequestID: "req1",
|
|
ManualRequestTime: time.Unix(123, 0),
|
|
ManualTimeout: time.Minute,
|
|
ManualProcessed: true,
|
|
ManualEventID: "event1",
|
|
}
|
|
tpl.EventExtra = api.EventExtra{
|
|
EventManualRequestID: "req2",
|
|
EventWatermark: time.Unix(456, 0),
|
|
}
|
|
tpl.CreateTime = tpl.CreateTime.In(time.UTC)
|
|
require.Equal(t, *tpl, *record)
|
|
|
|
// tags full update again
|
|
err = store.Update(ctx, tpl.ID, &api.TimerUpdate{
|
|
Tags: api.NewOptionalVal([]string{"l3"}),
|
|
})
|
|
require.NoError(t, err)
|
|
record, err = store.GetByID(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
tpl.Version = record.Version
|
|
tpl.Tags = []string{"l3"}
|
|
require.Equal(t, *tpl, *record)
|
|
|
|
// update manual request
|
|
err = store.Update(ctx, tpl.ID, &api.TimerUpdate{
|
|
ManualRequest: api.NewOptionalVal(api.ManualRequest{
|
|
ManualRequestID: "req3",
|
|
}),
|
|
})
|
|
require.NoError(t, err)
|
|
record, err = store.GetByID(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
tpl.Version = record.Version
|
|
tpl.ManualRequest = api.ManualRequest{
|
|
ManualRequestID: "req3",
|
|
}
|
|
require.Equal(t, *tpl, *record)
|
|
|
|
// update event extra
|
|
err = store.Update(ctx, tpl.ID, &api.TimerUpdate{
|
|
EventExtra: api.NewOptionalVal(api.EventExtra{
|
|
EventManualRequestID: "req4",
|
|
}),
|
|
})
|
|
require.NoError(t, err)
|
|
record, err = store.GetByID(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
tpl.Version = record.Version
|
|
tpl.EventExtra = api.EventExtra{
|
|
EventManualRequestID: "req4",
|
|
}
|
|
require.Equal(t, *tpl, *record)
|
|
|
|
// set some to empty
|
|
var zeroTime time.Time
|
|
err = store.Update(ctx, tpl.ID, &api.TimerUpdate{
|
|
TimeZone: api.NewOptionalVal(""),
|
|
Tags: api.NewOptionalVal([]string(nil)),
|
|
ManualRequest: api.NewOptionalVal(api.ManualRequest{}),
|
|
EventStatus: api.NewOptionalVal(api.SchedEventIdle),
|
|
EventID: api.NewOptionalVal(""),
|
|
EventData: api.NewOptionalVal([]byte(nil)),
|
|
EventStart: api.NewOptionalVal(zeroTime),
|
|
EventExtra: api.NewOptionalVal(api.EventExtra{}),
|
|
Watermark: api.NewOptionalVal(zeroTime),
|
|
SummaryData: api.NewOptionalVal([]byte(nil)),
|
|
})
|
|
require.NoError(t, err)
|
|
record, err = store.GetByID(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
tpl.TimeZone = ""
|
|
tpl.Location = timeutil.SystemLocation()
|
|
tpl.Version = record.Version
|
|
tpl.Tags = nil
|
|
tpl.ManualRequest = api.ManualRequest{}
|
|
tpl.EventStatus = api.SchedEventIdle
|
|
tpl.EventID = ""
|
|
tpl.EventData = nil
|
|
tpl.EventStart = zeroTime
|
|
tpl.EventExtra = api.EventExtra{}
|
|
tpl.Watermark = zeroTime
|
|
tpl.SummaryData = nil
|
|
tpl.CreateTime = tpl.CreateTime.In(tpl.Location)
|
|
require.Equal(t, *tpl, *record)
|
|
|
|
// err check version
|
|
err = store.Update(ctx, tpl.ID, &api.TimerUpdate{
|
|
SchedPolicyExpr: api.NewOptionalVal("2h"),
|
|
CheckVersion: api.NewOptionalVal(record.Version + 1),
|
|
})
|
|
require.EqualError(t, err, "timer version not match")
|
|
record, err = store.GetByID(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
require.Equal(t, *tpl, *record)
|
|
|
|
// err check event ID
|
|
err = store.Update(ctx, tpl.ID, &api.TimerUpdate{
|
|
SchedPolicyExpr: api.NewOptionalVal("2h"),
|
|
CheckEventID: api.NewOptionalVal("aabb"),
|
|
})
|
|
require.EqualError(t, err, "timer event id not match")
|
|
record, err = store.GetByID(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
require.Equal(t, *tpl, *record)
|
|
|
|
// err update
|
|
err = store.Update(ctx, tpl.ID, &api.TimerUpdate{
|
|
SchedPolicyExpr: api.NewOptionalVal("2x"),
|
|
})
|
|
require.EqualError(t, err, "schedule event configuration is not valid: invalid schedule event expr '2x': unknown unit x")
|
|
record, err = store.GetByID(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
require.Equal(t, *tpl, *record)
|
|
|
|
err = store.Update(ctx, tpl.ID, &api.TimerUpdate{
|
|
TimeZone: api.NewOptionalVal("invalid"),
|
|
})
|
|
require.ErrorContains(t, err, "Unknown or incorrect time zone: 'invalid'")
|
|
record, err = store.GetByID(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
require.Equal(t, *tpl, *record)
|
|
|
|
err = store.Update(ctx, tpl.ID, &api.TimerUpdate{
|
|
TimeZone: api.NewOptionalVal("tidb"),
|
|
})
|
|
require.ErrorContains(t, err, "Unknown or incorrect time zone: 'tidb'")
|
|
record, err = store.GetByID(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
require.Equal(t, *tpl, *record)
|
|
}
|
|
|
|
func runTimerStoreDelete(ctx context.Context, t *testing.T, store *api.TimerStore, tpl *api.TimerRecord) {
|
|
exist, err := store.Delete(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
require.True(t, exist)
|
|
|
|
_, err = store.GetByID(ctx, tpl.ID)
|
|
require.True(t, errors.ErrorEqual(err, api.ErrTimerNotExist))
|
|
|
|
exist, err = store.Delete(ctx, tpl.ID)
|
|
require.NoError(t, err)
|
|
require.False(t, exist)
|
|
}
|
|
|
|
func runTimerStoreInsertAndList(ctx context.Context, t *testing.T, store *api.TimerStore) {
|
|
records, err := store.List(ctx, nil)
|
|
require.NoError(t, err)
|
|
require.Empty(t, records)
|
|
|
|
recordTpl1 := api.TimerRecord{
|
|
TimerSpec: api.TimerSpec{
|
|
Namespace: "n1",
|
|
Key: "/path/to/key1",
|
|
SchedPolicyType: api.SchedEventInterval,
|
|
SchedPolicyExpr: "1h",
|
|
},
|
|
EventStatus: api.SchedEventIdle,
|
|
}
|
|
|
|
recordTpl2 := api.TimerRecord{
|
|
TimerSpec: api.TimerSpec{
|
|
Namespace: "n1",
|
|
Key: "/path/to/key2",
|
|
SchedPolicyType: api.SchedEventInterval,
|
|
SchedPolicyExpr: "2h",
|
|
Tags: []string{"tag1", "tag2"},
|
|
},
|
|
EventStatus: api.SchedEventIdle,
|
|
}
|
|
|
|
recordTpl3 := api.TimerRecord{
|
|
TimerSpec: api.TimerSpec{
|
|
Namespace: "n2",
|
|
Key: "/path/to/another",
|
|
SchedPolicyType: api.SchedEventInterval,
|
|
SchedPolicyExpr: "3h",
|
|
Tags: []string{"tag2", "tag3"},
|
|
},
|
|
EventStatus: api.SchedEventIdle,
|
|
}
|
|
|
|
id, err := store.Create(ctx, &recordTpl1)
|
|
require.NoError(t, err)
|
|
got, err := store.GetByID(ctx, id)
|
|
require.NoError(t, err)
|
|
recordTpl1.ID = got.ID
|
|
recordTpl1.Location = timeutil.SystemLocation()
|
|
recordTpl1.Version = got.Version
|
|
recordTpl1.CreateTime = got.CreateTime
|
|
|
|
id, err = store.Create(ctx, &recordTpl2)
|
|
require.NoError(t, err)
|
|
got, err = store.GetByID(ctx, id)
|
|
require.NoError(t, err)
|
|
recordTpl2.ID = got.ID
|
|
recordTpl2.Version = got.Version
|
|
recordTpl2.Location = timeutil.SystemLocation()
|
|
recordTpl2.CreateTime = got.CreateTime
|
|
|
|
id, err = store.Create(ctx, &recordTpl3)
|
|
require.NoError(t, err)
|
|
got, err = store.GetByID(ctx, id)
|
|
require.NoError(t, err)
|
|
recordTpl3.ID = got.ID
|
|
recordTpl3.Version = got.Version
|
|
recordTpl3.Location = timeutil.SystemLocation()
|
|
recordTpl3.CreateTime = got.CreateTime
|
|
|
|
checkList := func(expected []*api.TimerRecord, list []*api.TimerRecord) {
|
|
expectedMap := make(map[string]*api.TimerRecord, len(expected))
|
|
for _, r := range expected {
|
|
expectedMap[r.ID] = r
|
|
}
|
|
|
|
for _, r := range list {
|
|
require.Contains(t, expectedMap, r.ID)
|
|
got, ok := expectedMap[r.ID]
|
|
require.True(t, ok)
|
|
require.Equal(t, *got, *r)
|
|
delete(expectedMap, r.ID)
|
|
}
|
|
|
|
require.Empty(t, expectedMap)
|
|
}
|
|
|
|
timers, err := store.List(ctx, nil)
|
|
require.NoError(t, err)
|
|
checkList([]*api.TimerRecord{&recordTpl1, &recordTpl2, &recordTpl3}, timers)
|
|
|
|
timers, err = store.List(ctx, &api.TimerCond{
|
|
Key: api.NewOptionalVal("/path/to/k"),
|
|
KeyPrefix: true,
|
|
})
|
|
require.NoError(t, err)
|
|
checkList([]*api.TimerRecord{&recordTpl1, &recordTpl2}, timers)
|
|
|
|
timers, err = store.List(ctx, &api.TimerCond{
|
|
Key: api.NewOptionalVal("/path/to/k"),
|
|
})
|
|
require.NoError(t, err)
|
|
checkList([]*api.TimerRecord{}, timers)
|
|
|
|
timers, err = store.List(ctx, &api.TimerCond{
|
|
Namespace: api.NewOptionalVal("n2"),
|
|
Key: api.NewOptionalVal("/path/to/key2"),
|
|
})
|
|
require.NoError(t, err)
|
|
checkList([]*api.TimerRecord{}, timers)
|
|
|
|
timers, err = store.List(ctx, &api.TimerCond{
|
|
Namespace: api.NewOptionalVal("n1"),
|
|
Key: api.NewOptionalVal("/path/to/key2"),
|
|
})
|
|
require.NoError(t, err)
|
|
checkList([]*api.TimerRecord{&recordTpl2}, timers)
|
|
|
|
timers, err = store.List(ctx, &api.TimerCond{
|
|
Tags: api.NewOptionalVal([]string{"tag2"}),
|
|
})
|
|
require.NoError(t, err)
|
|
checkList([]*api.TimerRecord{&recordTpl2, &recordTpl3}, timers)
|
|
|
|
timers, err = store.List(ctx, &api.TimerCond{
|
|
Tags: api.NewOptionalVal([]string{"tag1", "tag3"}),
|
|
})
|
|
require.NoError(t, err)
|
|
checkList([]*api.TimerRecord{}, timers)
|
|
|
|
timers, err = store.List(ctx, &api.TimerCond{
|
|
Tags: api.NewOptionalVal([]string{"tag2", "tag3"}),
|
|
})
|
|
require.NoError(t, err)
|
|
checkList([]*api.TimerRecord{&recordTpl3}, timers)
|
|
|
|
timers, err = store.List(ctx, api.And(
|
|
&api.TimerCond{Namespace: api.NewOptionalVal("n1")},
|
|
&api.TimerCond{Tags: api.NewOptionalVal([]string{"tag2"})},
|
|
))
|
|
require.NoError(t, err)
|
|
checkList([]*api.TimerRecord{&recordTpl2}, timers)
|
|
|
|
timers, err = store.List(ctx, api.Not(api.And(
|
|
&api.TimerCond{Namespace: api.NewOptionalVal("n1")},
|
|
&api.TimerCond{Tags: api.NewOptionalVal([]string{"tag2"})},
|
|
)))
|
|
require.NoError(t, err)
|
|
checkList([]*api.TimerRecord{&recordTpl1, &recordTpl3}, timers)
|
|
|
|
timers, err = store.List(ctx, api.Or(
|
|
&api.TimerCond{Key: api.NewOptionalVal("/path/to/key2")},
|
|
&api.TimerCond{Tags: api.NewOptionalVal([]string{"tag3"})},
|
|
))
|
|
require.NoError(t, err)
|
|
checkList([]*api.TimerRecord{&recordTpl2, &recordTpl3}, timers)
|
|
|
|
timers, err = store.List(ctx, api.Not(api.Or(
|
|
&api.TimerCond{Key: api.NewOptionalVal("/path/to/key2")},
|
|
&api.TimerCond{Tags: api.NewOptionalVal([]string{"tag3"})},
|
|
)))
|
|
require.NoError(t, err)
|
|
checkList([]*api.TimerRecord{&recordTpl1}, timers)
|
|
}
|
|
|
|
func runTimerStoreWatchTest(t *testing.T, store *api.TimerStore) {
|
|
require.True(t, store.WatchSupported())
|
|
ctx, cancel := context.WithCancel(context.Background())
|
|
defer func() {
|
|
cancel()
|
|
}()
|
|
|
|
timer := api.TimerRecord{
|
|
TimerSpec: api.TimerSpec{
|
|
Namespace: "n1",
|
|
Key: "/path/to/key",
|
|
SchedPolicyType: api.SchedEventInterval,
|
|
SchedPolicyExpr: "1h",
|
|
Data: []byte("data1"),
|
|
},
|
|
}
|
|
|
|
ch := store.Watch(ctx)
|
|
assertWatchEvent := func(tp api.WatchTimerEventType, id string) {
|
|
timeout := time.NewTimer(time.Minute)
|
|
defer timeout.Stop()
|
|
select {
|
|
case resp, ok := <-ch:
|
|
if id == "" {
|
|
require.False(t, ok)
|
|
return
|
|
}
|
|
require.True(t, ok)
|
|
require.NotNil(t, resp)
|
|
require.Equal(t, 1, len(resp.Events))
|
|
require.Equal(t, tp, resp.Events[0].Tp)
|
|
require.Equal(t, id, resp.Events[0].TimerID)
|
|
case <-timeout.C:
|
|
require.FailNow(t, "no response")
|
|
}
|
|
}
|
|
|
|
id, err := store.Create(ctx, &timer)
|
|
require.NoError(t, err)
|
|
assertWatchEvent(api.WatchTimerEventCreate, id)
|
|
|
|
err = store.Update(ctx, id, &api.TimerUpdate{
|
|
SchedPolicyExpr: api.NewOptionalVal("2h"),
|
|
})
|
|
require.NoError(t, err)
|
|
assertWatchEvent(api.WatchTimerEventUpdate, id)
|
|
|
|
exit, err := store.Delete(ctx, id)
|
|
require.NoError(t, err)
|
|
require.True(t, exit)
|
|
assertWatchEvent(api.WatchTimerEventDelete, id)
|
|
|
|
cancel()
|
|
assertWatchEvent(0, "")
|
|
}
|
|
|
|
func TestMemNotifier(t *testing.T) {
|
|
notifier := api.NewMemTimerWatchEventNotifier()
|
|
defer notifier.Close()
|
|
runNotifierTest(t, notifier)
|
|
}
|
|
|
|
type multiNotifier struct {
|
|
notifier1 api.TimerWatchEventNotifier
|
|
notifier2 api.TimerWatchEventNotifier
|
|
}
|
|
|
|
func (n *multiNotifier) Notify(tp api.WatchTimerEventType, timerID string) {
|
|
n.notifier1.Notify(tp, timerID)
|
|
}
|
|
|
|
func (n *multiNotifier) Watch(ctx context.Context) api.WatchTimerChan {
|
|
return n.notifier2.Watch(ctx)
|
|
}
|
|
|
|
func (n *multiNotifier) Close() {
|
|
n.notifier1.Close()
|
|
n.notifier2.Close()
|
|
}
|
|
|
|
func TestEtcdNotifier(t *testing.T) {
|
|
integration.BeforeTestExternal(t)
|
|
testEtcdCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
|
defer testEtcdCluster.Terminate(t)
|
|
|
|
cli := testEtcdCluster.RandClient()
|
|
notifier := tablestore.NewEtcdNotifier(1, cli)
|
|
defer notifier.Close()
|
|
runNotifierTest(t, notifier)
|
|
|
|
// test one notifier notify, the other one watch
|
|
notifier = &multiNotifier{
|
|
notifier1: tablestore.NewEtcdNotifier(1, cli),
|
|
notifier2: tablestore.NewEtcdNotifier(1, cli),
|
|
}
|
|
defer notifier.Close()
|
|
runNotifierTest(t, notifier)
|
|
}
|
|
|
|
func runNotifierTest(t *testing.T, notifier api.TimerWatchEventNotifier) {
|
|
defer notifier.Close()
|
|
|
|
checkWatcherEvents := func(ch api.WatchTimerChan, events []api.WatchTimerEvent) {
|
|
gotEvents := make([]api.WatchTimerEvent, 0, len(events))
|
|
loop:
|
|
for {
|
|
select {
|
|
case <-time.After(time.Minute):
|
|
require.Equal(t, events, gotEvents, "wait events timeout")
|
|
return
|
|
case resp, ok := <-ch:
|
|
if !ok {
|
|
break loop
|
|
}
|
|
|
|
require.NotEmpty(t, resp.Events)
|
|
for _, event := range resp.Events {
|
|
gotEvents = append(gotEvents, *event)
|
|
}
|
|
if len(gotEvents) >= len(events) {
|
|
break loop
|
|
}
|
|
}
|
|
}
|
|
require.Equal(t, events, gotEvents)
|
|
}
|
|
|
|
checkWatcherClosed := func(ch api.WatchTimerChan, checkNoData bool) {
|
|
for {
|
|
select {
|
|
case _, ok := <-ch:
|
|
if !ok {
|
|
return
|
|
}
|
|
require.False(t, checkNoData)
|
|
case <-time.After(time.Minute):
|
|
require.FailNow(t, "wait closed timeout")
|
|
}
|
|
}
|
|
}
|
|
|
|
ctx1, cancel1 := context.WithCancel(context.Background())
|
|
defer cancel1()
|
|
watcher1 := notifier.Watch(ctx1)
|
|
|
|
ctx2, cancel2 := context.WithCancel(context.Background())
|
|
defer cancel2()
|
|
watcher2 := notifier.Watch(ctx2)
|
|
|
|
time.Sleep(time.Second)
|
|
notifier.Notify(api.WatchTimerEventCreate, "1")
|
|
notifier.Notify(api.WatchTimerEventCreate, "2")
|
|
notifier.Notify(api.WatchTimerEventUpdate, "1")
|
|
notifier.Notify(api.WatchTimerEventDelete, "2")
|
|
|
|
expectedEvents := []api.WatchTimerEvent{
|
|
{
|
|
Tp: api.WatchTimerEventCreate,
|
|
TimerID: "1",
|
|
},
|
|
{
|
|
Tp: api.WatchTimerEventCreate,
|
|
TimerID: "2",
|
|
},
|
|
{
|
|
Tp: api.WatchTimerEventUpdate,
|
|
TimerID: "1",
|
|
},
|
|
{
|
|
Tp: api.WatchTimerEventDelete,
|
|
TimerID: "2",
|
|
},
|
|
}
|
|
checkWatcherEvents(watcher1, expectedEvents)
|
|
checkWatcherEvents(watcher2, expectedEvents)
|
|
notifier.Notify(api.WatchTimerEventCreate, "3")
|
|
notifier.Notify(api.WatchTimerEventUpdate, "3")
|
|
cancel1()
|
|
notifier.Notify(api.WatchTimerEventDelete, "3")
|
|
notifier.Notify(api.WatchTimerEventCreate, "4")
|
|
expectedEvents = []api.WatchTimerEvent{
|
|
{
|
|
Tp: api.WatchTimerEventCreate,
|
|
TimerID: "3",
|
|
},
|
|
{
|
|
Tp: api.WatchTimerEventUpdate,
|
|
TimerID: "3",
|
|
},
|
|
{
|
|
Tp: api.WatchTimerEventDelete,
|
|
TimerID: "3",
|
|
},
|
|
{
|
|
Tp: api.WatchTimerEventCreate,
|
|
TimerID: "4",
|
|
},
|
|
}
|
|
checkWatcherClosed(watcher1, false)
|
|
checkWatcherEvents(watcher2, expectedEvents)
|
|
notifier.Notify(api.WatchTimerEventCreate, "5")
|
|
notifier.Close()
|
|
watcher3 := notifier.Watch(context.Background())
|
|
time.Sleep(time.Second)
|
|
notifier.Notify(api.WatchTimerEventDelete, "4")
|
|
watcher4 := notifier.Watch(context.Background())
|
|
time.Sleep(time.Second)
|
|
checkWatcherClosed(watcher2, false)
|
|
checkWatcherClosed(watcher3, true)
|
|
checkWatcherClosed(watcher4, true)
|
|
}
|
|
|
|
type mockHook struct {
|
|
preFunc func(ctx context.Context, event api.TimerShedEvent) (api.PreSchedEventResult, error)
|
|
schedFunc func(ctx context.Context, event api.TimerShedEvent) error
|
|
}
|
|
|
|
func (h *mockHook) Start() {}
|
|
|
|
func (h *mockHook) Stop() {}
|
|
|
|
func (h *mockHook) OnPreSchedEvent(ctx context.Context, event api.TimerShedEvent) (r api.PreSchedEventResult, err error) {
|
|
if h.preFunc != nil {
|
|
return h.preFunc(ctx, event)
|
|
}
|
|
return
|
|
}
|
|
|
|
func (h *mockHook) OnSchedEvent(ctx context.Context, event api.TimerShedEvent) error {
|
|
if h.schedFunc != nil {
|
|
return h.schedFunc(ctx, event)
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func TestTableStoreManualTrigger(t *testing.T) {
|
|
store, do := testkit.CreateMockStoreAndDomain(t)
|
|
tk := testkit.NewTestKit(t, store)
|
|
dbName := "test"
|
|
tblName := "timerstore"
|
|
tk.MustExec("use test")
|
|
tk.MustExec(createTimerTableSQL(dbName, tblName))
|
|
|
|
timerStore := tablestore.NewTableTimerStore(1, do.AdvancedSysSessionPool(), dbName, tblName, nil)
|
|
defer timerStore.Close()
|
|
|
|
var hookReqID atomic.Pointer[string]
|
|
hook := &mockHook{
|
|
preFunc: func(ctx context.Context, event api.TimerShedEvent) (r api.PreSchedEventResult, err error) {
|
|
timer := event.Timer()
|
|
require.False(t, timer.ManualProcessed)
|
|
require.Empty(t, timer.ManualEventID)
|
|
return
|
|
},
|
|
schedFunc: func(ctx context.Context, event api.TimerShedEvent) error {
|
|
timer := event.Timer()
|
|
require.Equal(t, timer.ManualRequestID, timer.EventManualRequestID)
|
|
require.Equal(t, timer.Watermark.Unix(), timer.EventWatermark.Unix())
|
|
require.True(t, timer.ManualProcessed)
|
|
require.Equal(t, timer.EventID, timer.ManualEventID)
|
|
hookReqID.Store(&timer.EventManualRequestID)
|
|
return nil
|
|
},
|
|
}
|
|
|
|
rt := runtime.NewTimerRuntimeBuilder("test", timerStore).
|
|
RegisterHookFactory("hook1", func(hookClass string, cli api.TimerClient) api.Hook { return hook }).
|
|
Build()
|
|
|
|
rt.Start()
|
|
defer rt.Stop()
|
|
|
|
cli := api.NewDefaultTimerClient(timerStore)
|
|
timer, err := cli.CreateTimer(context.TODO(), api.TimerSpec{
|
|
Key: "key1",
|
|
HookClass: "hook1",
|
|
SchedPolicyType: api.SchedEventInterval,
|
|
SchedPolicyExpr: "1h",
|
|
Watermark: time.Now(),
|
|
Enable: true,
|
|
})
|
|
require.NoError(t, err)
|
|
|
|
reqID, err := cli.ManualTriggerEvent(context.TODO(), timer.ID)
|
|
require.NoError(t, err)
|
|
start := time.Now()
|
|
eventID := ""
|
|
for eventID == "" {
|
|
if time.Since(start) > time.Minute {
|
|
require.FailNow(t, "timeout")
|
|
}
|
|
time.Sleep(100 * time.Millisecond)
|
|
timer, err = cli.GetTimerByID(context.TODO(), timer.ID)
|
|
require.NoError(t, err)
|
|
require.Equal(t, reqID, timer.ManualRequestID)
|
|
eventID = timer.EventID
|
|
}
|
|
|
|
require.Equal(t, reqID, timer.ManualRequestID)
|
|
require.Equal(t, eventID, timer.ManualEventID)
|
|
require.True(t, timer.ManualProcessed)
|
|
require.Equal(t, reqID, timer.EventManualRequestID)
|
|
start = time.Now()
|
|
for hookReqID.Load() == nil {
|
|
if time.Since(start) > time.Minute {
|
|
require.FailNow(t, "timeout")
|
|
}
|
|
time.Sleep(100 * time.Microsecond)
|
|
}
|
|
require.Equal(t, reqID, *hookReqID.Load())
|
|
|
|
require.NoError(t, cli.CloseTimerEvent(context.TODO(), timer.ID, timer.EventID))
|
|
timer, err = cli.GetTimerByID(context.TODO(), timer.ID)
|
|
require.NoError(t, err)
|
|
require.Equal(t, reqID, timer.ManualRequestID)
|
|
require.Equal(t, eventID, timer.ManualEventID)
|
|
require.True(t, timer.ManualProcessed)
|
|
require.Equal(t, api.EventExtra{}, timer.EventExtra)
|
|
}
|
|
|
|
func TestTimerStoreWithTimeZone(t *testing.T) {
|
|
// mem store
|
|
testTimerStoreWithTimeZone(t, api.NewMemoryTimerStore(), timeutil.SystemLocation().String())
|
|
|
|
// table store
|
|
store, do := testkit.CreateMockStoreAndDomain(t)
|
|
tk := testkit.NewTestKit(t, store)
|
|
dbName := "test"
|
|
tblName := "timerstore"
|
|
tk.MustExec("use test")
|
|
tk.MustExec(createTimerTableSQL(dbName, tblName))
|
|
tk.MustExec("set @@time_zone = 'America/Los_Angeles'")
|
|
|
|
pool := &mockSessionPool{t: t, pool: do.AdvancedSysSessionPool()}
|
|
timerStore := tablestore.NewTableTimerStore(1, pool, dbName, tblName, nil)
|
|
defer timerStore.Close()
|
|
|
|
testTimerStoreWithTimeZone(t, timerStore, timeutil.SystemLocation().String())
|
|
tk.MustExec("set @@global.time_zone='Asia/Tokyo'")
|
|
tk.MustExec(fmt.Sprintf("truncate table %s.%s", dbName, tblName))
|
|
testTimerStoreWithTimeZone(t, timerStore, "Asia/Tokyo")
|
|
|
|
// check time zone should be set back to the previous one.
|
|
require.Equal(t, "America/Los_Angeles", tk.Session().GetSessionVars().Location().String())
|
|
|
|
// check pool
|
|
require.False(t, pool.inuse.Load())
|
|
}
|
|
|
|
func testTimerStoreWithTimeZone(t *testing.T, timerStore *api.TimerStore, defaultTZ string) {
|
|
// 2024-11-03 09:30:00 UTC is 2024-11-03 01:30:00 -08:00 in `America/Los_Angeles`
|
|
// We should notice that it should not be regarded as 2024-11-03 01:30:00 -07:00
|
|
// because of DST these two times have the same format in time zone `America/Los_Angeles`.
|
|
time1, err := time.ParseInLocation(time.DateTime, "2024-11-03 09:30:00", time.UTC)
|
|
require.NoError(t, err)
|
|
|
|
time2, err := time.ParseInLocation(time.DateTime, "2024-11-03 08:30:00", time.UTC)
|
|
require.NoError(t, err)
|
|
|
|
id1, err := timerStore.Create(context.TODO(), &api.TimerRecord{
|
|
TimerSpec: api.TimerSpec{
|
|
Namespace: "default",
|
|
Key: "test1",
|
|
SchedPolicyType: api.SchedEventInterval,
|
|
SchedPolicyExpr: "1h",
|
|
Watermark: time1,
|
|
},
|
|
EventStatus: api.SchedEventTrigger,
|
|
EventStart: time2,
|
|
})
|
|
require.NoError(t, err)
|
|
|
|
id2, err := timerStore.Create(context.TODO(), &api.TimerRecord{
|
|
TimerSpec: api.TimerSpec{
|
|
Namespace: "default",
|
|
Key: "test2",
|
|
SchedPolicyType: api.SchedEventInterval,
|
|
SchedPolicyExpr: "1h",
|
|
Watermark: time2,
|
|
},
|
|
EventStatus: api.SchedEventTrigger,
|
|
EventStart: time1,
|
|
})
|
|
require.NoError(t, err)
|
|
|
|
// create case
|
|
timer1, err := timerStore.GetByID(context.TODO(), id1)
|
|
require.NoError(t, err)
|
|
require.Equal(t, time1.In(time.UTC).String(), timer1.Watermark.In(time.UTC).String())
|
|
require.Equal(t, time2.In(time.UTC).String(), timer1.EventStart.In(time.UTC).String())
|
|
checkTimerRecordLocation(t, timer1, defaultTZ)
|
|
|
|
timer2, err := timerStore.GetByID(context.TODO(), id2)
|
|
require.NoError(t, err)
|
|
require.Equal(t, time2.In(time.UTC).String(), timer2.Watermark.In(time.UTC).String())
|
|
require.Equal(t, time1.In(time.UTC).String(), timer2.EventStart.In(time.UTC).String())
|
|
checkTimerRecordLocation(t, timer2, defaultTZ)
|
|
|
|
// update time
|
|
require.NoError(t, timerStore.Update(context.TODO(), id1, &api.TimerUpdate{
|
|
Watermark: api.NewOptionalVal(time2),
|
|
EventStart: api.NewOptionalVal(time1),
|
|
}))
|
|
|
|
require.NoError(t, timerStore.Update(context.TODO(), id2, &api.TimerUpdate{
|
|
Watermark: api.NewOptionalVal(time1),
|
|
EventStart: api.NewOptionalVal(time2),
|
|
}))
|
|
|
|
timer1, err = timerStore.GetByID(context.TODO(), id1)
|
|
require.NoError(t, err)
|
|
require.Equal(t, time2.In(time.UTC).String(), timer1.Watermark.In(time.UTC).String())
|
|
require.Equal(t, time1.In(time.UTC).String(), timer1.EventStart.In(time.UTC).String())
|
|
checkTimerRecordLocation(t, timer1, defaultTZ)
|
|
|
|
timer2, err = timerStore.GetByID(context.TODO(), id2)
|
|
require.NoError(t, err)
|
|
require.Equal(t, time1.In(time.UTC).String(), timer2.Watermark.In(time.UTC).String())
|
|
require.Equal(t, time2.In(time.UTC).String(), timer2.EventStart.In(time.UTC).String())
|
|
checkTimerRecordLocation(t, timer2, defaultTZ)
|
|
|
|
// update timezone
|
|
require.NoError(t, timerStore.Update(context.TODO(), id1, &api.TimerUpdate{
|
|
TimeZone: api.NewOptionalVal("Europe/Berlin"),
|
|
}))
|
|
|
|
timer1, err = timerStore.GetByID(context.TODO(), id1)
|
|
require.NoError(t, err)
|
|
require.Equal(t, time2.In(time.UTC).String(), timer1.Watermark.In(time.UTC).String())
|
|
require.Equal(t, time1.In(time.UTC).String(), timer1.EventStart.In(time.UTC).String())
|
|
checkTimerRecordLocation(t, timer1, "Europe/Berlin")
|
|
|
|
require.NoError(t, timerStore.Update(context.TODO(), id1, &api.TimerUpdate{
|
|
TimeZone: api.NewOptionalVal(""),
|
|
}))
|
|
|
|
timer1, err = timerStore.GetByID(context.TODO(), id1)
|
|
require.NoError(t, err)
|
|
require.Equal(t, time2.In(time.UTC).String(), timer1.Watermark.In(time.UTC).String())
|
|
require.Equal(t, time1.In(time.UTC).String(), timer1.EventStart.In(time.UTC).String())
|
|
checkTimerRecordLocation(t, timer1, defaultTZ)
|
|
}
|
|
|
|
func checkTimerRecordLocation(t *testing.T, r *api.TimerRecord, tz string) {
|
|
require.Equal(t, tz, r.Location.String())
|
|
require.Same(t, r.Location, r.Watermark.Location())
|
|
require.Same(t, r.Location, r.CreateTime.Location())
|
|
if !r.EventStart.IsZero() {
|
|
require.Same(t, r.Location, r.EventStart.Location())
|
|
}
|
|
}
|