Skip to content

Commit

Permalink
enhance: using streaming service in insert/upsert/flush/delete/queryn…
Browse files Browse the repository at this point in the history
…ode (milvus-io#35406)

issue: milvus-io#33285

- using streaming service in insert/upsert/flush/delete/querynode
- fixup flusher bugs and refactor the flush operation
- enable streaming service for dml and ddl
- pass the e2e when enabling streaming service
- pass the integration tst when enabling streaming service

---------

Signed-off-by: chyezh <chyezh@outlook.com>
Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
  • Loading branch information
chyezh authored Aug 29, 2024
1 parent ae53ce3 commit 99dff06
Show file tree
Hide file tree
Showing 102 changed files with 2,737 additions and 837 deletions.
2 changes: 1 addition & 1 deletion Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -537,7 +537,7 @@ generate-mockery-chunk-manager: getdeps
generate-mockery-pkg:
$(MAKE) -C pkg generate-mockery

generate-mockery-internal:
generate-mockery-internal: getdeps
$(INSTALL_PATH)/mockery --config $(PWD)/internal/.mockery.yaml

generate-mockery: generate-mockery-types generate-mockery-kv generate-mockery-rootcoord generate-mockery-proxy generate-mockery-querycoord generate-mockery-querynode generate-mockery-datacoord generate-mockery-pkg generate-mockery-internal
Expand Down
8 changes: 8 additions & 0 deletions cmd/roles/roles.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,12 +35,14 @@ import (

"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/cmd/components"
"github.com/milvus-io/milvus/internal/distributed/streaming"
"github.com/milvus-io/milvus/internal/http"
"github.com/milvus-io/milvus/internal/http/healthz"
"github.com/milvus-io/milvus/internal/util/dependency"
kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv"
"github.com/milvus-io/milvus/internal/util/initcore"
internalmetrics "github.com/milvus-io/milvus/internal/util/metrics"
"github.com/milvus-io/milvus/internal/util/streamingutil"
"github.com/milvus-io/milvus/pkg/config"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
Expand Down Expand Up @@ -377,6 +379,12 @@ func (mr *MilvusRoles) Run() {
paramtable.SetRole(mr.ServerType)
}

// Initialize streaming service if enabled.
if streamingutil.IsStreamingServiceEnabled() {
streaming.Init()
defer streaming.Release()
}

expr.Init()
expr.Register("param", paramtable.Get())
mr.setupLogger()
Expand Down
5 changes: 5 additions & 0 deletions cmd/tools/config/generate.go
Original file line number Diff line number Diff line change
Expand Up @@ -328,6 +328,11 @@ func WriteYaml(w io.Writer) {
#milvus will automatically initialize half of the available GPU memory,
#maxMemSize will the whole available GPU memory.`,
},
{
name: "streamingNode",
header: `
# Any configuration related to the streaming node server.`,
},
}
marshller := YamlMarshaller{w, groups, result}
marshller.writeYamlRecursive(lo.Filter(result, func(d DocContent, _ int) bool {
Expand Down
10 changes: 10 additions & 0 deletions configs/milvus.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -1034,3 +1034,13 @@ trace:
gpu:
initMemSize: 2048 # Gpu Memory Pool init size
maxMemSize: 4096 # Gpu Memory Pool Max size

# Any configuration related to the streaming node server.
streamingNode:
ip: # TCP/IP address of streamingNode. If not specified, use the first unicastable address
port: 22222 # TCP port of streamingNode
grpc:
serverMaxSendSize: 268435456 # The maximum size of each RPC request that the streamingNode can send, unit: byte
serverMaxRecvSize: 268435456 # The maximum size of each RPC request that the streamingNode can receive, unit: byte
clientMaxSendSize: 268435456 # The maximum size of each RPC request that the clients on streamingNode can send, unit: byte
clientMaxRecvSize: 268435456 # The maximum size of each RPC request that the clients on streamingNode can receive, unit: byte
15 changes: 12 additions & 3 deletions docker-compose.yml
Original file line number Diff line number Diff line change
Expand Up @@ -97,14 +97,23 @@ services:
- ETCD_QUOTA_BACKEND_BYTES=4294967296
- ETCD_SNAPSHOT_COUNT=50000
healthcheck:
test: ['CMD', '/opt/bitnami/scripts/etcd/healthcheck.sh']
test: [ 'CMD', '/opt/bitnami/scripts/etcd/healthcheck.sh' ]
interval: 30s
timeout: 20s
retries: 3

pulsar:
image: apachepulsar/pulsar:2.8.2
command: bin/pulsar standalone --no-functions-worker --no-stream-storage
command: |
/bin/bash -c \
"bin/apply-config-from-env.py conf/standalone.conf && \
exec bin/pulsar standalone --no-functions-worker --no-stream-storage"
environment:
# 10MB
- PULSAR_PREFIX_maxMessageSize=10485760
# this is 104857600 + 10240 (padding)
- nettyMaxFrameSizeBytes=104867840
- PULSAR_GC=-XX:+UseG1GC

minio:
image: minio/minio:RELEASE.2023-03-20T20-16-18Z
Expand All @@ -113,7 +122,7 @@ services:
MINIO_SECRET_KEY: minioadmin
command: minio server /minio_data
healthcheck:
test: ['CMD', 'curl', '-f', 'http://localhost:9000/minio/health/live']
test: [ 'CMD', 'curl', '-f', 'http://localhost:9000/minio/health/live' ]
interval: 30s
timeout: 20s
retries: 3
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ require (
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0
github.com/klauspost/compress v1.17.7
github.com/mgutz/ansi v0.0.0-20200706080929-d51e80ef957d
github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240815123953-6dab6fcd6454
github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240820032106-b34be93a2271
github.com/minio/minio-go/v7 v7.0.61
github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81
github.com/prometheus/client_golang v1.14.0
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -598,8 +598,8 @@ github.com/milvus-io/cgosymbolizer v0.0.0-20240722103217-b7dee0e50119 h1:9VXijWu
github.com/milvus-io/cgosymbolizer v0.0.0-20240722103217-b7dee0e50119/go.mod h1:DvXTE/K/RtHehxU8/GtDs4vFtfw64jJ3PaCnFri8CRg=
github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b h1:TfeY0NxYxZzUfIfYe5qYDBzt4ZYRqzUjTR6CvUzjat8=
github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b/go.mod h1:iwW+9cWfIzzDseEBCCeDSN5SD16Tidvy8cwQ7ZY8Qj4=
github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240815123953-6dab6fcd6454 h1:JmZCYjMPpiE4ksZw0AUxXWkDY7wwA4fhS+SO1N211Vw=
github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240815123953-6dab6fcd6454/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs=
github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240820032106-b34be93a2271 h1:YUWBgtRHmvkxMPTfOrY3FIq0K5XHw02Z18z7cyaMH04=
github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240820032106-b34be93a2271/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs=
github.com/milvus-io/pulsar-client-go v0.6.10 h1:eqpJjU+/QX0iIhEo3nhOqMNXL+TyInAs1IAHZCrCM/A=
github.com/milvus-io/pulsar-client-go v0.6.10/go.mod h1:lQqCkgwDF8YFYjKA+zOheTk1tev2B+bKj5j7+nm8M1w=
github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpspGNG7Z948v4n35fFGB3RR3G/ry4FWs=
Expand Down
9 changes: 8 additions & 1 deletion internal/datacoord/channel_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (

"github.com/milvus-io/milvus/internal/datacoord/allocator"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/streamingutil"
"github.com/milvus-io/milvus/pkg/kv"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/conc"
Expand Down Expand Up @@ -160,7 +161,7 @@ func (m *ChannelManagerImpl) Startup(ctx context.Context, legacyNodes, allNodes
m.finishRemoveChannel(info.NodeID, lo.Values(info.Channels)...)
}

if m.balanceCheckLoop != nil {
if m.balanceCheckLoop != nil && !streamingutil.IsStreamingServiceEnabled() {
log.Info("starting channel balance loop")
m.wg.Add(1)
go func() {
Expand Down Expand Up @@ -329,6 +330,12 @@ func (m *ChannelManagerImpl) Balance() {
}

func (m *ChannelManagerImpl) Match(nodeID UniqueID, channel string) bool {
if streamingutil.IsStreamingServiceEnabled() {
// Skip the channel matching check since the
// channel manager no longer manages channels in streaming mode.
return true
}

m.mu.RLock()
defer m.mu.RUnlock()

Expand Down
5 changes: 4 additions & 1 deletion internal/datacoord/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -724,7 +724,10 @@ func (s *Server) startServerLoop() {
go s.importScheduler.Start()
go s.importChecker.Start()
s.garbageCollector.start()
s.syncSegmentsScheduler.Start()

if !streamingutil.IsStreamingServiceEnabled() {
s.syncSegmentsScheduler.Start()
}
}

func (s *Server) updateSegmentStatistics(stats []*commonpb.SegmentStats) {
Expand Down
71 changes: 41 additions & 30 deletions internal/datacoord/services.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
"github.com/milvus-io/milvus/internal/util/componentutil"
"github.com/milvus-io/milvus/internal/util/importutilv2"
"github.com/milvus-io/milvus/internal/util/segmentutil"
"github.com/milvus-io/milvus/internal/util/streamingutil"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
Expand Down Expand Up @@ -111,14 +112,16 @@ func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.F
}
timeOfSeal, _ := tsoutil.ParseTS(ts)

sealedSegmentIDs, err := s.segmentManager.SealAllSegments(ctx, req.GetCollectionID(), req.GetSegmentIDs())
if err != nil {
return &datapb.FlushResponse{
Status: merr.Status(errors.Wrapf(err, "failed to flush collection %d",
req.GetCollectionID())),
}, nil
sealedSegmentIDs := make([]int64, 0)
if !streamingutil.IsStreamingServiceEnabled() {
var err error
if sealedSegmentIDs, err = s.segmentManager.SealAllSegments(ctx, req.GetCollectionID(), req.GetSegmentIDs()); err != nil {
return &datapb.FlushResponse{
Status: merr.Status(errors.Wrapf(err, "failed to flush collection %d",
req.GetCollectionID())),
}, nil
}
}

sealedSegmentsIDDict := make(map[UniqueID]bool)
for _, sealedSegmentID := range sealedSegmentIDs {
sealedSegmentsIDDict[sealedSegmentID] = true
Expand All @@ -135,33 +138,35 @@ func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.F
}
}

var isUnimplemented bool
err = retry.Do(ctx, func() error {
nodeChannels := s.channelManager.GetNodeChannelsByCollectionID(req.GetCollectionID())
if !streamingutil.IsStreamingServiceEnabled() {
var isUnimplemented bool
err = retry.Do(ctx, func() error {
nodeChannels := s.channelManager.GetNodeChannelsByCollectionID(req.GetCollectionID())

for nodeID, channelNames := range nodeChannels {
err = s.cluster.FlushChannels(ctx, nodeID, ts, channelNames)
if err != nil && errors.Is(err, merr.ErrServiceUnimplemented) {
isUnimplemented = true
return nil
}
if err != nil {
return err
for nodeID, channelNames := range nodeChannels {
err = s.cluster.FlushChannels(ctx, nodeID, ts, channelNames)
if err != nil && errors.Is(err, merr.ErrServiceUnimplemented) {
isUnimplemented = true
return nil
}
if err != nil {
return err
}
}
return nil
}, retry.Attempts(60)) // about 3min
if err != nil {
return &datapb.FlushResponse{
Status: merr.Status(err),
}, nil
}
return nil
}, retry.Attempts(60)) // about 3min
if err != nil {
return &datapb.FlushResponse{
Status: merr.Status(err),
}, nil
}

if isUnimplemented {
// For compatible with rolling upgrade from version 2.2.x,
// fall back to the flush logic of version 2.2.x;
log.Warn("DataNode FlushChannels unimplemented", zap.Error(err))
ts = 0
if isUnimplemented {
// For compatible with rolling upgrade from version 2.2.x,
// fall back to the flush logic of version 2.2.x;
log.Warn("DataNode FlushChannels unimplemented", zap.Error(err))
ts = 0
}
}

log.Info("flush response with segments",
Expand Down Expand Up @@ -255,6 +260,12 @@ func (s *Server) AllocSegment(ctx context.Context, req *datapb.AllocSegmentReque
return &datapb.AllocSegmentResponse{Status: merr.Status(merr.ErrParameterInvalid)}, nil
}

// refresh the meta of the collection.
_, err := s.handler.GetCollection(ctx, req.GetCollectionId())
if err != nil {
return &datapb.AllocSegmentResponse{Status: merr.Status(err)}, nil
}

// Alloc new growing segment and return the segment info.
segmentInfo, err := s.segmentManager.AllocNewGrowingSegment(ctx, req.GetCollectionId(), req.GetPartitionId(), req.GetSegmentId(), req.GetVchannel())
if err != nil {
Expand Down
21 changes: 12 additions & 9 deletions internal/datanode/data_node.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ import (
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/streamingutil"
"github.com/milvus-io/milvus/pkg/kv"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
Expand Down Expand Up @@ -308,20 +309,22 @@ func (node *DataNode) Start() error {
return
}

node.writeBufferManager.Start()
if !streamingutil.IsStreamingServiceEnabled() {
node.writeBufferManager.Start()

go node.compactionExecutor.Start(node.ctx)
node.timeTickSender = util2.NewTimeTickSender(node.broker, node.session.ServerID,
retry.Attempts(20), retry.Sleep(time.Millisecond*100))
node.timeTickSender.Start()

go node.importScheduler.Start()
node.channelManager = channel.NewChannelManager(getPipelineParams(node), node.flowgraphManager)
node.channelManager.Start()

node.timeTickSender = util2.NewTimeTickSender(node.broker, node.session.ServerID,
retry.Attempts(20), retry.Sleep(time.Millisecond*100))
node.timeTickSender.Start()
go node.channelCheckpointUpdater.Start()
}

go node.channelCheckpointUpdater.Start()
go node.compactionExecutor.Start(node.ctx)

node.channelManager = channel.NewChannelManager(getPipelineParams(node), node.flowgraphManager)
node.channelManager.Start()
go node.importScheduler.Start()

node.UpdateStateCode(commonpb.StateCode_Healthy)
})
Expand Down
6 changes: 3 additions & 3 deletions internal/distributed/streaming/internal/errs/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@ import (

// All error in streamingservice package should be marked by streamingservice/errs package.
var (
ErrClosed = errors.New("closed")
ErrCanceled = errors.New("canceled")
ErrTxnUnavailable = errors.New("transaction unavailable")
ErrClosed = errors.New("closed")
ErrCanceledOrDeadlineExceed = errors.New("canceled or deadline exceed")
ErrUnrecoverable = errors.New("unrecoverable")
)
9 changes: 5 additions & 4 deletions internal/distributed/streaming/internal/producer/producer.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,13 +95,14 @@ func (p *ResumableProducer) Produce(ctx context.Context, msg message.MutableMess
}
// It's ok to stop retry if the error is canceled or deadline exceed.
if status.IsCanceled(err) {
return nil, errors.Mark(err, errs.ErrCanceled)
return nil, errors.Mark(err, errs.ErrCanceledOrDeadlineExceed)
}
if sErr := status.AsStreamingError(err); sErr != nil {
// if the error is txn unavailable, it cannot be retried forever.
// if the error is txn unavailable or unrecoverable error,
// it cannot be retried forever.
// we should mark it and return.
if sErr.IsTxnUnavilable() {
return nil, errors.Mark(err, errs.ErrTxnUnavailable)
if sErr.IsUnrecoverable() {
return nil, errors.Mark(err, errs.ErrUnrecoverable)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ func (p *producerWithResumingError) GetProducerAfterAvailable(ctx context.Contex
p.cond.L.Lock()
for p.err == nil && (p.producer == nil || !p.producer.IsAvailable()) {
if err := p.cond.Wait(ctx); err != nil {
return nil, errors.Mark(err, errs.ErrCanceled)
return nil, errors.Mark(err, errs.ErrCanceledOrDeadlineExceed)
}
}
err := p.err
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ func TestResumableProducer(t *testing.T) {
id, err = rp.Produce(ctx, msg)
assert.Nil(t, id)
assert.Error(t, err)
assert.True(t, errors.Is(err, errs.ErrCanceled))
assert.True(t, errors.Is(err, errs.ErrCanceledOrDeadlineExceed))

// Test the underlying handler close.
close(ch2)
Expand Down
33 changes: 14 additions & 19 deletions internal/distributed/streaming/streaming.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,14 +80,24 @@ type WALAccesser interface {
// Once the txn is returned, the Commit or Rollback operation must be called once, otherwise resource leak on wal.
Txn(ctx context.Context, opts TxnOption) (Txn, error)

// Append writes a records to the log.
Append(ctx context.Context, msgs message.MutableMessage, opts ...AppendOption) (*types.AppendResult, error)
// RawAppend writes a records to the log.
RawAppend(ctx context.Context, msgs message.MutableMessage, opts ...AppendOption) (*types.AppendResult, error)

// Read returns a scanner for reading records from the wal.
Read(ctx context.Context, opts ReadOption) Scanner

// Utility returns the utility for writing records to the log.
Utility() Utility
// AppendMessages appends messages to the wal.
// It it a helper utility function to append messages to the wal.
// If the messages is belong to one vchannel, it will be sent as a transaction.
// Otherwise, it will be sent as individual messages.
// !!! This function do not promise the atomicity and deliver order of the messages appending.
// TODO: Remove after we support cross-wal txn.
AppendMessages(ctx context.Context, msgs ...message.MutableMessage) AppendResponses

// AppendMessagesWithOption appends messages to the wal with the given option.
// Same with AppendMessages, but with the given option.
// TODO: Remove after we support cross-wal txn.
AppendMessagesWithOption(ctx context.Context, opts AppendOption, msgs ...message.MutableMessage) AppendResponses
}

// Txn is the interface for writing transaction into the wal.
Expand All @@ -105,18 +115,3 @@ type Txn interface {
// It is preserved for future cross-wal txn.
Rollback(ctx context.Context) error
}

type Utility interface {
// AppendMessages appends messages to the wal.
// It it a helper utility function to append messages to the wal.
// If the messages is belong to one vchannel, it will be sent as a transaction.
// Otherwise, it will be sent as individual messages.
// !!! This function do not promise the atomicity and deliver order of the messages appending.
// TODO: Remove after we support cross-wal txn.
AppendMessages(ctx context.Context, msgs ...message.MutableMessage) AppendResponses

// AppendMessagesWithOption appends messages to the wal with the given option.
// Same with AppendMessages, but with the given option.
// TODO: Remove after we support cross-wal txn.
AppendMessagesWithOption(ctx context.Context, opts AppendOption, msgs ...message.MutableMessage) AppendResponses
}
Loading

0 comments on commit 99dff06

Please sign in to comment.