github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/logtail/service/client.go (about) 1 // Copyright 2021 Matrix Origin 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package service 16 17 import ( 18 "context" 19 "sync" 20 21 "go.uber.org/ratelimit" 22 "go.uber.org/zap" 23 24 "github.com/matrixorigin/matrixone/pkg/common/moerr" 25 "github.com/matrixorigin/matrixone/pkg/common/morpc" 26 "github.com/matrixorigin/matrixone/pkg/logutil" 27 "github.com/matrixorigin/matrixone/pkg/pb/api" 28 "github.com/matrixorigin/matrixone/pkg/pb/logtail" 29 v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" 30 ) 31 32 type ClientOption func(*LogtailClient) 33 34 func WithClientRequestPerSecond(rps int) ClientOption { 35 return func(c *LogtailClient) { 36 c.options.rps = rps 37 } 38 } 39 40 // LogtailClient encapsulates morpc stream. 41 type LogtailClient struct { 42 stream morpc.Stream 43 recvChan chan morpc.Message 44 broken chan struct{} // mark morpc stream as broken when necessary 45 once sync.Once 46 47 options struct { 48 rps int 49 } 50 51 limiter ratelimit.Limiter 52 } 53 54 // NewLogtailClient constructs LogtailClient. 55 func NewLogtailClient(stream morpc.Stream, opts ...ClientOption) (*LogtailClient, error) { 56 client := &LogtailClient{ 57 stream: stream, 58 broken: make(chan struct{}), 59 } 60 61 recvChan, err := stream.Receive() 62 if err != nil { 63 logutil.Error("logtail client: fail to fetch message channel from morpc stream", zap.Error(err)) 64 return nil, err 65 } 66 client.recvChan = recvChan 67 68 client.options.rps = 200 69 for _, opt := range opts { 70 opt(client) 71 } 72 client.limiter = ratelimit.New(client.options.rps) 73 74 return client, nil 75 } 76 77 // Close closes stream. 78 func (c *LogtailClient) Close() error { 79 err := c.stream.Close(true) 80 if err != nil { 81 logutil.Error("logtail client: fail to close morpc stream", zap.Error(err)) 82 } 83 return err 84 } 85 86 // Subscribe subscribes table. 87 func (c *LogtailClient) Subscribe( 88 ctx context.Context, table api.TableID, 89 ) error { 90 if c.streamBroken() { 91 logutil.Error("logtail client: subscribe via broken morpc stream") 92 return moerr.NewStreamClosedNoCtx() 93 } 94 95 c.limiter.Take() 96 97 request := &LogtailRequest{} 98 request.Request = &logtail.LogtailRequest_SubscribeTable{ 99 SubscribeTable: &logtail.SubscribeRequest{ 100 Table: &table, 101 }, 102 } 103 request.SetID(c.stream.ID()) 104 105 err := c.stream.Send(ctx, request) 106 if err != nil { 107 logutil.Error("logtail client: fail to subscribe via morpc stream", zap.Error(err)) 108 } 109 return err 110 } 111 112 // Unsubscribe cancel subscription for table. 113 func (c *LogtailClient) Unsubscribe( 114 ctx context.Context, table api.TableID, 115 ) error { 116 if c.streamBroken() { 117 logutil.Error("logtail client: unsubscribe via broken morpc stream") 118 return moerr.NewStreamClosedNoCtx() 119 } 120 121 c.limiter.Take() 122 123 request := &LogtailRequest{} 124 request.Request = &logtail.LogtailRequest_UnsubscribeTable{ 125 UnsubscribeTable: &logtail.UnsubscribeRequest{ 126 Table: &table, 127 }, 128 } 129 request.SetID(c.stream.ID()) 130 err := c.stream.Send(ctx, request) 131 if err != nil { 132 logutil.Error("logtail client: fail to unsubscribe via morpc stream", zap.Error(err)) 133 } 134 return err 135 } 136 137 // Receive fetches logtail response. 138 // 139 // 1. response for error: *LogtailResponse.GetError() != nil 140 // 2. response for subscription: *LogtailResponse.GetSubscribeResponse() != nil 141 // 3. response for unsubscription: *LogtailResponse.GetUnsubscribeResponse() != nil 142 // 3. response for incremental logtail: *LogtailResponse.GetUpdateResponse() != nil 143 func (c *LogtailClient) Receive(ctx context.Context) (*LogtailResponse, error) { 144 recvFunc := func() (*LogtailResponseSegment, error) { 145 select { 146 case <-ctx.Done(): 147 return nil, ctx.Err() 148 149 case <-c.broken: 150 return nil, moerr.NewStreamClosedNoCtx() 151 152 case message, ok := <-c.recvChan: 153 if !ok || message == nil { 154 logutil.Error("logtail client: morpc stream broken", 155 zap.Bool("is message nil", message == nil), 156 zap.Bool("is message channel closed", !ok), 157 ) 158 159 // mark stream as broken 160 c.once.Do(func() { close(c.broken) }) 161 return nil, moerr.NewStreamClosedNoCtx() 162 } 163 v2.LogTailReceiveQueueSizeGauge.Set(float64(len(c.recvChan))) 164 return message.(*LogtailResponseSegment), nil 165 } 166 } 167 168 prev, err := recvFunc() 169 if err != nil { 170 return nil, err 171 } 172 buf := make([]byte, 0, prev.MessageSize) 173 buf = AppendChunk(buf, prev.GetPayload()) 174 175 for prev.Sequence < prev.MaxSequence { 176 segment, err := recvFunc() 177 if err != nil { 178 return nil, err 179 } 180 buf = AppendChunk(buf, segment.GetPayload()) 181 prev = segment 182 } 183 184 resp := &LogtailResponse{} 185 if err := resp.Unmarshal(buf); err != nil { 186 logutil.Error("logtail client: fail to unmarshal logtail response", zap.Error(err)) 187 return nil, err 188 } 189 return resp, nil 190 } 191 192 // streamBroken returns true if stream is borken. 193 func (c *LogtailClient) streamBroken() bool { 194 select { 195 case <-c.broken: 196 return true 197 default: 198 } 199 return false 200 }