mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
fix: streaming consume checkpoint is always nil and limit resource of ci (#39781)
issue: #38399 - fix the nil pointer bug - limit the resource usage for streaming e2e - enhance the go test - fix: rootcoord block when graceful stop --------- Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
parent
5e6f6af65d
commit
034575396f
@ -347,6 +347,12 @@ func (s *Server) Stop() (err error) {
|
|||||||
defer s.tikvCli.Close()
|
defer s.tikvCli.Close()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if s.rootCoord != nil {
|
||||||
|
log.Info("graceful stop rootCoord")
|
||||||
|
s.rootCoord.GracefulStop()
|
||||||
|
log.Info("graceful stop rootCoord done")
|
||||||
|
}
|
||||||
|
|
||||||
if s.grpcServer != nil {
|
if s.grpcServer != nil {
|
||||||
utils.GracefulStopGRPCServer(s.grpcServer)
|
utils.GracefulStopGRPCServer(s.grpcServer)
|
||||||
}
|
}
|
||||||
|
|||||||
@ -118,6 +118,9 @@ func (m *mockCore) Stop() error {
|
|||||||
return fmt.Errorf("stop error")
|
return fmt.Errorf("stop error")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *mockCore) GracefulStop() {
|
||||||
|
}
|
||||||
|
|
||||||
func TestRun(t *testing.T) {
|
func TestRun(t *testing.T) {
|
||||||
paramtable.Init()
|
paramtable.Init()
|
||||||
parameters := []string{"tikv", "etcd"}
|
parameters := []string{"tikv", "etcd"}
|
||||||
|
|||||||
@ -110,7 +110,7 @@ func (c *catalog) GetConsumeCheckpoint(ctx context.Context, pchannelName string)
|
|||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
val := &streamingpb.WALCheckpoint{}
|
val := &streamingpb.WALCheckpoint{}
|
||||||
if err = proto.Unmarshal([]byte(value), &streamingpb.WALCheckpoint{}); err != nil {
|
if err = proto.Unmarshal([]byte(value), val); err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
return val, nil
|
return val, nil
|
||||||
|
|||||||
@ -4,15 +4,52 @@ import (
|
|||||||
"context"
|
"context"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/cockroachdb/errors"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/stretchr/testify/mock"
|
"github.com/stretchr/testify/mock"
|
||||||
"google.golang.org/protobuf/proto"
|
"google.golang.org/protobuf/proto"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/kv/mocks"
|
"github.com/milvus-io/milvus/internal/kv/mocks"
|
||||||
"github.com/milvus-io/milvus/pkg/proto/streamingpb"
|
"github.com/milvus-io/milvus/pkg/proto/streamingpb"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestCatalog(t *testing.T) {
|
func TestCatalogConsumeCheckpoint(t *testing.T) {
|
||||||
|
kv := mocks.NewMetaKv(t)
|
||||||
|
v := streamingpb.WALCheckpoint{}
|
||||||
|
vs, err := proto.Marshal(&v)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
kv.EXPECT().Load(mock.Anything, mock.Anything).Return(string(vs), nil)
|
||||||
|
catalog := NewCataLog(kv)
|
||||||
|
ctx := context.Background()
|
||||||
|
checkpoint, err := catalog.GetConsumeCheckpoint(ctx, "p1")
|
||||||
|
assert.NotNil(t, checkpoint)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
kv.EXPECT().Load(mock.Anything, mock.Anything).Unset()
|
||||||
|
kv.EXPECT().Load(mock.Anything, mock.Anything).Return("", errors.New("err"))
|
||||||
|
checkpoint, err = catalog.GetConsumeCheckpoint(ctx, "p1")
|
||||||
|
assert.Nil(t, checkpoint)
|
||||||
|
assert.Error(t, err)
|
||||||
|
|
||||||
|
kv.EXPECT().Load(mock.Anything, mock.Anything).Unset()
|
||||||
|
kv.EXPECT().Load(mock.Anything, mock.Anything).Return("", merr.ErrIoKeyNotFound)
|
||||||
|
checkpoint, err = catalog.GetConsumeCheckpoint(ctx, "p1")
|
||||||
|
assert.Nil(t, checkpoint)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
kv.EXPECT().Save(mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
||||||
|
err = catalog.SaveConsumeCheckpoint(ctx, "p1", &streamingpb.WALCheckpoint{})
|
||||||
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
kv.EXPECT().Save(mock.Anything, mock.Anything, mock.Anything).Unset()
|
||||||
|
kv.EXPECT().Save(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("err"))
|
||||||
|
err = catalog.SaveConsumeCheckpoint(ctx, "p1", &streamingpb.WALCheckpoint{})
|
||||||
|
assert.Error(t, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestCatalogSegmentAssignments(t *testing.T) {
|
||||||
kv := mocks.NewMetaKv(t)
|
kv := mocks.NewMetaKv(t)
|
||||||
k := "p1"
|
k := "p1"
|
||||||
v := streamingpb.SegmentAssignmentMeta{}
|
v := streamingpb.SegmentAssignmentMeta{}
|
||||||
|
|||||||
@ -826,15 +826,18 @@ func (c *Core) revokeSession() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (c *Core) GracefulStop() {
|
||||||
|
if c.streamingCoord != nil {
|
||||||
|
c.streamingCoord.Stop()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// Stop stops rootCoord.
|
// Stop stops rootCoord.
|
||||||
func (c *Core) Stop() error {
|
func (c *Core) Stop() error {
|
||||||
c.UpdateStateCode(commonpb.StateCode_Abnormal)
|
c.UpdateStateCode(commonpb.StateCode_Abnormal)
|
||||||
c.stopExecutor()
|
c.stopExecutor()
|
||||||
c.stopScheduler()
|
c.stopScheduler()
|
||||||
|
|
||||||
if c.streamingCoord != nil {
|
|
||||||
c.streamingCoord.Stop()
|
|
||||||
}
|
|
||||||
if c.proxyWatcher != nil {
|
if c.proxyWatcher != nil {
|
||||||
c.proxyWatcher.Stop()
|
c.proxyWatcher.Stop()
|
||||||
}
|
}
|
||||||
|
|||||||
@ -67,7 +67,8 @@ func (r *resumingWatcher) Close() {
|
|||||||
|
|
||||||
func (r *resumingWatcher) execute(backoffConfig *typeutil.BackoffTimerConfig) {
|
func (r *resumingWatcher) execute(backoffConfig *typeutil.BackoffTimerConfig) {
|
||||||
backoff := typeutil.NewBackoffTimer(backoffConfig)
|
backoff := typeutil.NewBackoffTimer(backoffConfig)
|
||||||
nextTimer := time.After(0)
|
var nextTimer <-chan time.Time
|
||||||
|
var initialized bool
|
||||||
var watcher Watcher
|
var watcher Watcher
|
||||||
defer func() {
|
defer func() {
|
||||||
if watcher != nil {
|
if watcher != nil {
|
||||||
@ -92,6 +93,12 @@ func (r *resumingWatcher) execute(backoffConfig *typeutil.BackoffTimerConfig) {
|
|||||||
watcher = nil
|
watcher = nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if !initialized {
|
||||||
|
// try to initialize watcher in next loop.
|
||||||
|
// avoid to make a grpc stream channel if the watch operation is not used.
|
||||||
|
nextTimer = time.After(0)
|
||||||
|
initialized = true
|
||||||
|
}
|
||||||
case ev, ok := <-eventChan:
|
case ev, ok := <-eventChan:
|
||||||
if !ok {
|
if !ok {
|
||||||
watcher.Close()
|
watcher.Close()
|
||||||
@ -101,15 +108,15 @@ func (r *resumingWatcher) execute(backoffConfig *typeutil.BackoffTimerConfig) {
|
|||||||
r.evs.Notify(ev)
|
r.evs.Notify(ev)
|
||||||
case <-nextTimer:
|
case <-nextTimer:
|
||||||
var err error
|
var err error
|
||||||
|
nextTimer = nil
|
||||||
if watcher, err = r.createNewWatcher(); err != nil {
|
if watcher, err = r.createNewWatcher(); err != nil {
|
||||||
r.Logger().Warn("create new watcher failed", zap.Error(err))
|
r.Logger().Warn("create new watcher failed", zap.Error(err))
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
r.Logger().Info("create new watcher successful")
|
r.Logger().Info("create new watcher successful")
|
||||||
backoff.DisableBackoff()
|
backoff.DisableBackoff()
|
||||||
nextTimer = nil
|
|
||||||
}
|
}
|
||||||
if watcher == nil {
|
if watcher == nil && nextTimer == nil {
|
||||||
backoff.EnableBackoff()
|
backoff.EnableBackoff()
|
||||||
var interval time.Duration
|
var interval time.Duration
|
||||||
nextTimer, interval = backoff.NextTimer()
|
nextTimer, interval = backoff.NextTimer()
|
||||||
|
|||||||
@ -65,7 +65,8 @@ func (b *balancerImpl) WatchChannelAssignments(ctx context.Context, cb func(vers
|
|||||||
}
|
}
|
||||||
defer b.lifetime.Done()
|
defer b.lifetime.Done()
|
||||||
|
|
||||||
ctx, _ = contextutil.MergeContext(ctx, b.ctx)
|
ctx, cancel := contextutil.MergeContext(ctx, b.ctx)
|
||||||
|
defer cancel()
|
||||||
return b.channelMetaManager.WatchAssignmentResult(ctx, cb)
|
return b.channelMetaManager.WatchAssignmentResult(ctx, cb)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -75,6 +76,8 @@ func (b *balancerImpl) MarkAsUnavailable(ctx context.Context, pChannels []types.
|
|||||||
}
|
}
|
||||||
defer b.lifetime.Done()
|
defer b.lifetime.Done()
|
||||||
|
|
||||||
|
ctx, cancel := contextutil.MergeContext(ctx, b.ctx)
|
||||||
|
defer cancel()
|
||||||
return b.sendRequestAndWaitFinish(ctx, newOpMarkAsUnavailable(ctx, pChannels))
|
return b.sendRequestAndWaitFinish(ctx, newOpMarkAsUnavailable(ctx, pChannels))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -85,6 +88,8 @@ func (b *balancerImpl) Trigger(ctx context.Context) error {
|
|||||||
}
|
}
|
||||||
defer b.lifetime.Done()
|
defer b.lifetime.Done()
|
||||||
|
|
||||||
|
ctx, cancel := contextutil.MergeContext(ctx, b.ctx)
|
||||||
|
defer cancel()
|
||||||
return b.sendRequestAndWaitFinish(ctx, newOpTrigger(ctx))
|
return b.sendRequestAndWaitFinish(ctx, newOpTrigger(ctx))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -43,7 +43,9 @@ func TestPChannelCheckpointManager(t *testing.T) {
|
|||||||
|
|
||||||
p.AddVChannel("vchannel-999", rmq.NewRmqID(1000000))
|
p.AddVChannel("vchannel-999", rmq.NewRmqID(1000000))
|
||||||
p.DropVChannel("vchannel-1000")
|
p.DropVChannel("vchannel-1000")
|
||||||
p.Update(vchannel, rmq.NewRmqID(1000001))
|
for _, vchannel := range vchannel {
|
||||||
|
p.Update(vchannel, rmq.NewRmqID(1000001))
|
||||||
|
}
|
||||||
|
|
||||||
assert.Eventually(t, func() bool {
|
assert.Eventually(t, func() bool {
|
||||||
newMinimum := minimumOne.Load()
|
newMinimum := minimumOne.Load()
|
||||||
|
|||||||
@ -12,7 +12,7 @@ import (
|
|||||||
)
|
)
|
||||||
|
|
||||||
func TestVChannelCheckpointManager(t *testing.T) {
|
func TestVChannelCheckpointManager(t *testing.T) {
|
||||||
exists, vchannel, minimumX := generateRandomExistsMessageID()
|
exists, vchannels, minimumX := generateRandomExistsMessageID()
|
||||||
m := newVChannelCheckpointManager(exists)
|
m := newVChannelCheckpointManager(exists)
|
||||||
assert.True(t, m.MinimumCheckpoint().EQ(minimumX))
|
assert.True(t, m.MinimumCheckpoint().EQ(minimumX))
|
||||||
|
|
||||||
@ -32,17 +32,31 @@ func TestVChannelCheckpointManager(t *testing.T) {
|
|||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
assert.True(t, m.MinimumCheckpoint().EQ(minimumX))
|
assert.True(t, m.MinimumCheckpoint().EQ(minimumX))
|
||||||
|
|
||||||
err = m.Update(vchannel, rmq.NewRmqID(1000001))
|
for _, vchannel := range vchannels {
|
||||||
assert.NoError(t, err)
|
err = m.Update(vchannel, rmq.NewRmqID(1000001))
|
||||||
|
assert.NoError(t, err)
|
||||||
|
}
|
||||||
assert.False(t, m.MinimumCheckpoint().EQ(minimumX))
|
assert.False(t, m.MinimumCheckpoint().EQ(minimumX))
|
||||||
|
|
||||||
err = m.Update(vchannel, minimumX)
|
err = m.Update(vchannels[0], minimumX)
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
|
|
||||||
err = m.Drop("vchannel-501")
|
err = m.Drop("vchannel-501")
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
lastMinimum := m.MinimumCheckpoint()
|
||||||
|
for i := 0; i < 1001; i++ {
|
||||||
|
m.Update(fmt.Sprintf("vchannel-%d", i), rmq.NewRmqID(rand.Int63n(9999999)+2))
|
||||||
|
newMinimum := m.MinimumCheckpoint()
|
||||||
|
assert.True(t, lastMinimum.LTE(newMinimum))
|
||||||
|
lastMinimum = newMinimum
|
||||||
|
}
|
||||||
for i := 0; i < 1001; i++ {
|
for i := 0; i < 1001; i++ {
|
||||||
m.Drop(fmt.Sprintf("vchannel-%d", i))
|
m.Drop(fmt.Sprintf("vchannel-%d", i))
|
||||||
|
newMinimum := m.MinimumCheckpoint()
|
||||||
|
if newMinimum != nil {
|
||||||
|
assert.True(t, lastMinimum.LTE(newMinimum))
|
||||||
|
lastMinimum = newMinimum
|
||||||
|
}
|
||||||
}
|
}
|
||||||
assert.Len(t, m.index, 0)
|
assert.Len(t, m.index, 0)
|
||||||
assert.Len(t, m.checkpointHeap, 0)
|
assert.Len(t, m.checkpointHeap, 0)
|
||||||
@ -50,17 +64,21 @@ func TestVChannelCheckpointManager(t *testing.T) {
|
|||||||
assert.Nil(t, m.MinimumCheckpoint())
|
assert.Nil(t, m.MinimumCheckpoint())
|
||||||
}
|
}
|
||||||
|
|
||||||
func generateRandomExistsMessageID() (map[string]message.MessageID, string, message.MessageID) {
|
func generateRandomExistsMessageID() (map[string]message.MessageID, []string, message.MessageID) {
|
||||||
minimumX := int64(10000000)
|
minimumX := int64(10000000)
|
||||||
var vchannel string
|
var vchannel []string
|
||||||
exists := make(map[string]message.MessageID)
|
exists := make(map[string]message.MessageID)
|
||||||
for i := 0; i < 1000; i++ {
|
for i := 0; i < 1000; i++ {
|
||||||
x := rand.Int63n(999999) + 2
|
x := rand.Int63n(999999) + 2
|
||||||
exists[fmt.Sprintf("vchannel-%d", i)] = rmq.NewRmqID(x)
|
exists[fmt.Sprintf("vchannel-%d", i)] = rmq.NewRmqID(x)
|
||||||
if x < minimumX {
|
if x < minimumX {
|
||||||
minimumX = x
|
minimumX = x
|
||||||
vchannel = fmt.Sprintf("vchannel-%d", i)
|
vchannel = []string{fmt.Sprintf("vchannel-%d", i)}
|
||||||
|
} else if x == minimumX {
|
||||||
|
vchannel = append(vchannel, fmt.Sprintf("vchannel-%d", i))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
vchannel = append(vchannel, "vchannel-1")
|
||||||
|
exists["vchannel-1"] = rmq.NewRmqID(minimumX)
|
||||||
return exists, vchannel, rmq.NewRmqID(minimumX)
|
return exists, vchannel, rmq.NewRmqID(minimumX)
|
||||||
}
|
}
|
||||||
|
|||||||
@ -213,6 +213,8 @@ type RootCoordComponent interface {
|
|||||||
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
|
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
|
||||||
|
|
||||||
RegisterStreamingCoordGRPCService(server *grpc.Server)
|
RegisterStreamingCoordGRPCService(server *grpc.Server)
|
||||||
|
|
||||||
|
GracefulStop()
|
||||||
}
|
}
|
||||||
|
|
||||||
// ProxyClient is the client interface for proxy server
|
// ProxyClient is the client interface for proxy server
|
||||||
|
|||||||
@ -20,7 +20,7 @@ dataCoordinator:
|
|||||||
dataNode:
|
dataNode:
|
||||||
resources:
|
resources:
|
||||||
limits:
|
limits:
|
||||||
cpu: "2"
|
cpu: "1"
|
||||||
requests:
|
requests:
|
||||||
cpu: "0.5"
|
cpu: "0.5"
|
||||||
memory: 500Mi
|
memory: 500Mi
|
||||||
@ -249,7 +249,21 @@ queryNode:
|
|||||||
cpu: "2"
|
cpu: "2"
|
||||||
requests:
|
requests:
|
||||||
cpu: "0.5"
|
cpu: "0.5"
|
||||||
memory: 500Mi
|
memory: 512Mi
|
||||||
|
streamingNode:
|
||||||
|
resources:
|
||||||
|
limits:
|
||||||
|
cpu: "2"
|
||||||
|
requests:
|
||||||
|
cpu: "0.5"
|
||||||
|
memory: 512Mi
|
||||||
|
mixCoordinator:
|
||||||
|
resources:
|
||||||
|
limits:
|
||||||
|
cpu: "1"
|
||||||
|
requests:
|
||||||
|
cpu: "0.2"
|
||||||
|
memory: 256Mi
|
||||||
rootCoordinator:
|
rootCoordinator:
|
||||||
resources:
|
resources:
|
||||||
limits:
|
limits:
|
||||||
|
|||||||
@ -153,6 +153,12 @@ func (chainTask *CollectionPrepare) CreateCollection(ctx context.Context, t *tes
|
|||||||
common.CheckErr(t, err, true)
|
common.CheckErr(t, err, true)
|
||||||
|
|
||||||
t.Cleanup(func() {
|
t.Cleanup(func() {
|
||||||
|
// The collection will be cleanup after the test
|
||||||
|
// But some ctx is setted with timeout for only a part of unittest,
|
||||||
|
// which will cause the drop collection failed with timeout.
|
||||||
|
ctx, cancel := context.WithTimeout(context.WithoutCancel(ctx), time.Second*10)
|
||||||
|
defer cancel()
|
||||||
|
|
||||||
err := mc.DropCollection(ctx, clientv2.NewDropCollectionOption(schema.CollectionName))
|
err := mc.DropCollection(ctx, clientv2.NewDropCollectionOption(schema.CollectionName))
|
||||||
common.CheckErr(t, err, true)
|
common.CheckErr(t, err, true)
|
||||||
})
|
})
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user