Skip to content

Commit

Permalink
[enhancement] logtail: make table subscription async (#16879)
Browse files Browse the repository at this point in the history
A table subscription takes long time if the table has much data
which need flushed, and it will block the logtail update of other
tables because they work in the same goroutine.

So we put the table subscription into a independent goroutine,
thus the logtails of other tables could be pushed to cn servers
without being blocked. And when the first phase of pulling table
data finishes, put it into the same goroutine with pushing-job,
and pull the data again for the second phase. At last, send the
whole data that are merged together to cn servers.

After the PR, the table subscription will not block pushing-jobs.
There are 50 parallel pulling jobs at the same time at most by
defalt to avoid OOM and other risks.

Approved by: @zhangxu19830126, @triump2020, @aptend, @XuPeng-SH
  • Loading branch information
volgariver6 authored Jun 18, 2024
1 parent 2e1afe3 commit 9504578
Show file tree
Hide file tree
Showing 17 changed files with 417 additions and 74 deletions.
1 change: 1 addition & 0 deletions pkg/tests/service/dnservice.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,6 +186,7 @@ func buildTNConfig(
cfg.LogtailServer.LogtailCollectInterval.Duration = opt.logtailPushServer.logtailCollectInterval
cfg.LogtailServer.LogtailRPCStreamPoisonTime.Duration = opt.logtailPushServer.logtailRPCStreamPoisonTIme
cfg.LogtailServer.LogtailResponseSendTimeout.Duration = opt.logtailPushServer.logtailResponseSendTimeout
cfg.LogtailServer.PullWorkerPoolSize = toml.ByteSize(opt.logtailPushServer.pullWorkerPoolSize)

// We need the filled version of configuration.
// It's necessary when building tnservice.Option.
Expand Down
11 changes: 11 additions & 0 deletions pkg/tests/service/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ const (
defaultRPCStreamPoisonTime = 5 * time.Second
defaultLogtailCollectInterval = 50 * time.Millisecond
defaultLogtailResponseSendTimeout = 10 * time.Second
defaultPullWorkerPoolSize = 50
)

// Options are params for creating test cluster.
Expand Down Expand Up @@ -111,6 +112,7 @@ type Options struct {
logtailRPCStreamPoisonTIme time.Duration
logtailCollectInterval time.Duration
logtailResponseSendTimeout time.Duration
pullWorkerPoolSize int64
}

cn struct {
Expand Down Expand Up @@ -204,6 +206,9 @@ func (opt *Options) validate() {
if opt.logtailPushServer.logtailResponseSendTimeout <= 0 {
opt.logtailPushServer.logtailResponseSendTimeout = defaultLogtailResponseSendTimeout
}
if opt.logtailPushServer.pullWorkerPoolSize <= 0 {
opt.logtailPushServer.pullWorkerPoolSize = defaultPullWorkerPoolSize
}
}

// BuildHAKeeperConfig returns hakeeper.Config
Expand Down Expand Up @@ -387,6 +392,12 @@ func (opt Options) WithLogtailResponseSendTimeout(timeout time.Duration) Options
return opt
}

// WithLogtailResponseSendTimeout sets response send timeout for logtail push server.
func (opt Options) WithPullWorkerPoolSize(s int64) Options {
opt.logtailPushServer.pullWorkerPoolSize = s
return opt
}

// WithCNOptionFunc set build cn options func
func (opt Options) WithCNOptionFunc(fn func(i int) []cnservice.Option) Options {
opt.cn.optionFunc = fn
Expand Down
8 changes: 8 additions & 0 deletions pkg/tnservice/cfg.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@ var (
defaultRPCStreamPoisonTime = 5 * time.Second
defaultLogtailCollectInterval = 2 * time.Millisecond
defaultLogtailResponseSendTimeout = 10 * time.Second
defaultPullWorkerPoolSize = 50

storageDir = "storage"
defaultDataDir = "./mo-data"
Expand Down Expand Up @@ -145,6 +146,7 @@ type Config struct {
LogtailRPCStreamPoisonTime toml.Duration `toml:"logtail-rpc-stream-poison-time"`
LogtailCollectInterval toml.Duration `toml:"logtail-collect-interval"`
LogtailResponseSendTimeout toml.Duration `toml:"logtail-response-send-timeout"`
PullWorkerPoolSize toml.ByteSize `toml:"pull-worker-pool-size"`
}

// Txn transactions configuration
Expand Down Expand Up @@ -275,6 +277,9 @@ func (c *Config) Validate() error {
if c.LogtailServer.LogtailResponseSendTimeout.Duration <= 0 {
c.LogtailServer.LogtailResponseSendTimeout.Duration = defaultLogtailResponseSendTimeout
}
if c.LogtailServer.PullWorkerPoolSize <= 0 {
c.LogtailServer.PullWorkerPoolSize = toml.ByteSize(defaultPullWorkerPoolSize)
}
if c.Cluster.RefreshInterval.Duration == 0 {
c.Cluster.RefreshInterval.Duration = time.Second * 10
}
Expand Down Expand Up @@ -384,6 +389,9 @@ func (c *Config) SetDefaultValue() {
if c.LogtailServer.LogtailResponseSendTimeout.Duration <= 0 {
c.LogtailServer.LogtailResponseSendTimeout.Duration = defaultLogtailResponseSendTimeout
}
if c.LogtailServer.PullWorkerPoolSize <= 0 {
c.LogtailServer.PullWorkerPoolSize = toml.ByteSize(defaultPullWorkerPoolSize)
}
if c.Cluster.RefreshInterval.Duration == 0 {
c.Cluster.RefreshInterval.Duration = time.Second * 10
}
Expand Down
1 change: 1 addition & 0 deletions pkg/tnservice/factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -167,6 +167,7 @@ func (s *store) newTAEStorage(ctx context.Context, shard metadata.TNShard, facto
RPCStreamPoisonTime: s.cfg.LogtailServer.LogtailRPCStreamPoisonTime.Duration,
LogtailCollectInterval: s.cfg.LogtailServer.LogtailCollectInterval.Duration,
ResponseSendTimeout: s.cfg.LogtailServer.LogtailResponseSendTimeout.Duration,
PullWorkerPoolSize: int64(s.cfg.LogtailServer.PullWorkerPoolSize),
}

// the previous values
Expand Down
2 changes: 2 additions & 0 deletions pkg/tnservice/service_ports_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ func TestService_RegisterServices(t *testing.T) {
LogtailRPCStreamPoisonTime toml.Duration `toml:"logtail-rpc-stream-poison-time"`
LogtailCollectInterval toml.Duration `toml:"logtail-collect-interval"`
LogtailResponseSendTimeout toml.Duration `toml:"logtail-response-send-timeout"`
PullWorkerPoolSize toml.ByteSize `toml:"pull-worker-pool-size"`
}(struct {
ListenAddress string
ServiceAddress string
Expand All @@ -57,6 +58,7 @@ func TestService_RegisterServices(t *testing.T) {
LogtailRPCStreamPoisonTime toml.Duration
LogtailCollectInterval toml.Duration
LogtailResponseSendTimeout toml.Duration
PullWorkerPoolSize toml.ByteSize
}{
ListenAddress: fmt.Sprintf("%s:%d", listenHost, port1+1),
ServiceAddress: fmt.Sprintf("%s:%d", serviceHost, port1+1),
Expand Down
37 changes: 34 additions & 3 deletions pkg/util/metric/v2/dashboard/grafana_dashboard_logtail.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ func (c *DashboardCreator) initLogTailDashboard() error {
c.initLogtailBytesRow(),
c.initLogtailLoadCheckpointRow(),
c.initLogtailCollectRow(),
c.initLogtailTransmitRow(),
c.initLogtailSubscriptionRow(),
c.initLogtailUpdatePartitionRow(),
)...)
Expand All @@ -49,10 +50,24 @@ func (c *DashboardCreator) initLogtailCollectRow() dashboard.Option {
return dashboard.Row(
"Logtail collect duration",
c.getHistogram(
"collect duration",
c.getMetricWithFilter("mo_logtail_collect_duration_seconds_bucket", ``),
"pull type phase1 collection duration",
c.getMetricWithFilter("mo_logtail_pull_collection_phase1_duration_seconds_bucket", ``),
[]float64{0.50, 0.8, 0.90, 0.99},
12,
4,
axis.Unit("s"),
axis.Min(0)),
c.getHistogram(
"pull type phase2 collection duration",
c.getMetricWithFilter("mo_logtail_pull_collection_phase2_duration_seconds_bucket", ``),
[]float64{0.50, 0.8, 0.90, 0.99},
4,
axis.Unit("s"),
axis.Min(0)),
c.getHistogram(
"push type collection duration",
c.getMetricWithFilter("mo_logtail_push_collection_duration_seconds_bucket", ``),
[]float64{0.50, 0.8, 0.90, 0.99},
4,
axis.Unit("s"),
axis.Min(0)),
)
Expand Down Expand Up @@ -213,3 +228,19 @@ func (c *DashboardCreator) initLogtailLoadCheckpointRow() dashboard.Option {
axis.Min(0)),
)
}

func (c *DashboardCreator) initLogtailTransmitRow() dashboard.Option {
return dashboard.Row(
"Logtail Transmit counter",
c.withGraph(
"Server Send",
6,
`sum(rate(`+c.getMetricWithFilter("mo_logtail_transmit_total", `type="server-send"`)+`[$interval]))`,
""),
c.withGraph(
"Client Receive",
6,
`sum(rate(`+c.getMetricWithFilter("mo_logtail_transmit_total", `type="client-receive"`)+`[$interval]))`,
""),
)
}
34 changes: 31 additions & 3 deletions pkg/util/metric/v2/logtail.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,14 +128,42 @@ var (
Buckets: getDurationBuckets(),
})

LogTailCollectDurationHistogram = prometheus.NewHistogram(
LogTailPullCollectionPhase1DurationHistogram = prometheus.NewHistogram(
prometheus.HistogramOpts{
Namespace: "mo",
Subsystem: "logtail",
Name: "collect_duration_seconds",
Help: "Bucketed histogram of logtail collecting duration.",
Name: "pull_collection_phase1_duration_seconds",
Help: "Bucketed histogram of logtail pull type collection duration of phase1.",
Buckets: getDurationBuckets(),
})

LogTailPullCollectionPhase2DurationHistogram = prometheus.NewHistogram(
prometheus.HistogramOpts{
Namespace: "mo",
Subsystem: "logtail",
Name: "pull_collection_phase2_duration_seconds",
Help: "Bucketed histogram of logtail pull type collection duration of phase2.",
Buckets: getDurationBuckets(),
})

LogTailPushCollectionDurationHistogram = prometheus.NewHistogram(
prometheus.HistogramOpts{
Namespace: "mo",
Subsystem: "logtail",
Name: "push_collection_duration_seconds",
Help: "Bucketed histogram of logtail push type collection duration.",
Buckets: getDurationBuckets(),
})

logTailTransmitCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "mo",
Subsystem: "logtail",
Name: "transmit_total",
Help: "Total number of transmit count.",
}, []string{"type"})
LogTailServerSendCounter = logTailTransmitCounter.WithLabelValues("server-send")
LogTailClientReceiveCounter = logTailTransmitCounter.WithLabelValues("client-receive")
)

var (
Expand Down
4 changes: 3 additions & 1 deletion pkg/util/metric/v2/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,9 @@ func initLogtailMetrics() {
registry.MustRegister(logTailSendDurationHistogram)
registry.MustRegister(LogTailLoadCheckpointDurationHistogram)

registry.MustRegister(LogTailCollectDurationHistogram)
registry.MustRegister(LogTailPushCollectionDurationHistogram)
registry.MustRegister(LogTailPullCollectionPhase1DurationHistogram)
registry.MustRegister(LogTailPullCollectionPhase2DurationHistogram)
registry.MustRegister(LogTailSubscriptionCounter)
registry.MustRegister(txnTNSideDurationHistogram)
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/vm/engine/disttae/logtail_consumer.go
Original file line number Diff line number Diff line change
Expand Up @@ -369,6 +369,9 @@ func (c *PushClient) receiveOneLogtail(ctx context.Context, e *Engine) error {
ctx, cancel := context.WithTimeout(ctx, maxTimeToWaitServerResponse)
defer cancel()

// Client receives one logtail counter.
defer v2.LogTailClientReceiveCounter.Add(1)

resp := c.subscriber.receiveResponse(ctx)
if resp.err != nil {
// POSSIBLE ERROR: context deadline exceeded, rpc closed, decode error.
Expand Down
96 changes: 96 additions & 0 deletions pkg/vm/engine/tae/logtail/service/merger.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,96 @@
// Copyright 2021 - 2024 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 service

import (
"fmt"

"github.com/matrixorigin/matrixone/pkg/pb/api"
"github.com/matrixorigin/matrixone/pkg/pb/logtail"
"github.com/matrixorigin/matrixone/pkg/pb/timestamp"
)

const (
ckpLocationDivider = ';'
)

// logtailMerger is the merger tool to merge multiple
// LogtailPhase instances.
type logtailMerger struct {
logtails []*LogtailPhase
}

func newLogtailMerger(l ...*LogtailPhase) *logtailMerger {
var lm logtailMerger
lm.logtails = append(lm.logtails, l...)
return &lm
}

// Merge merges all instances and return one logtail.TableLogtail
// and a callback function.
func (m *logtailMerger) Merge() (logtail.TableLogtail, func()) {
var tableID api.TableID
var entryCount int
var ts timestamp.Timestamp
for _, t := range m.logtails {
entryCount += len(t.tail.Commands)

// check the table ID.
if tableID.TbId == 0 {
tableID = *t.tail.Table
} else if tableID.TbId != t.tail.Table.TbId {
panic(fmt.Sprintf("cannot merge logtails with different table: %d, %d",
tableID.TbId, t.tail.Table.TbId))
}

// get the max timestamp.
if ts.Less(*t.tail.Ts) {
ts = *t.tail.Ts
}
}

// create the callbacks.
callbacks := make([]func(), 0, entryCount)
// create a new table logtail with the entry number.
tail := logtail.TableLogtail{
Ts: &ts,
Table: &tableID,
Commands: make([]api.Entry, 0, entryCount),
}
for _, t := range m.logtails {
ckpLocLen := len(tail.CkpLocation)
if ckpLocLen > 0 &&
tail.CkpLocation[ckpLocLen-1] != ckpLocationDivider {
tail.CkpLocation += string(ckpLocationDivider)
}
tail.CkpLocation += t.tail.CkpLocation
tail.Commands = append(tail.Commands, t.tail.Commands...)
callbacks = append(callbacks, t.closeCB)
}

// remove the last ';'
ckpLocLen := len(tail.CkpLocation)
if ckpLocLen > 0 &&
tail.CkpLocation[ckpLocLen-1] == ckpLocationDivider {
tail.CkpLocation = tail.CkpLocation[:ckpLocLen-1]
}
return tail, func() {
for _, cb := range callbacks {
if cb != nil {
cb()
}
}
}
}
68 changes: 68 additions & 0 deletions pkg/vm/engine/tae/logtail/service/merger_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
// Copyright 2021 - 2024 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 service

import (
"testing"

"github.com/matrixorigin/matrixone/pkg/pb/api"
"github.com/matrixorigin/matrixone/pkg/pb/logtail"
"github.com/matrixorigin/matrixone/pkg/pb/timestamp"
"github.com/stretchr/testify/assert"
)

func TestLogtailMerge(t *testing.T) {
var cbValue int
tail1 := &LogtailPhase{
tail: logtail.TableLogtail{
CkpLocation: "aaa;bbb;",
Ts: &timestamp.Timestamp{
PhysicalTime: 100,
},
Table: &api.TableID{
DbId: 1,
TbId: 2,
},
Commands: []api.Entry{{TableId: 2}},
},
closeCB: func() {
cbValue++
},
}
tail2 := &LogtailPhase{
tail: logtail.TableLogtail{
CkpLocation: "ccc;ddd",
Ts: &timestamp.Timestamp{
PhysicalTime: 200,
},
Table: &api.TableID{
DbId: 1,
TbId: 2,
},
Commands: []api.Entry{{TableId: 2}},
},
closeCB: func() {
cbValue++
},
}
lm := newLogtailMerger(tail1, tail2)
assert.NotNil(t, lm)
tail, cb := lm.Merge()
cb()
assert.Equal(t, 2, cbValue)
assert.Equal(t, 2, len(tail.Commands))
assert.Equal(t, int64(200), tail.Ts.PhysicalTime)
assert.Equal(t, "aaa;bbb;ccc;ddd", tail.CkpLocation)
}
Loading

0 comments on commit 9504578

Please sign in to comment.