mirror of
https://github.com/openimsdk/open-im-server.git
synced 2025-04-05 05:12:45 +08:00
feat: implement offline push using kafka (#2600)
* refactor: refactor workflows contents. * add tool workflows. * update field. * fix: remove chat error. * Fix err. * fix error. * remove cn comment. * update workflows files. * update infra config. * move workflows. * feat: update bot. * fix: solve uncorrect outdated msg get. * update get docIDs logic. * update * update skip logic. * fix * update. * fix: delay deleteObject func. * remove unused content. * update log type. * feat: implement request batch count limit. * update * update * feat: implement offline push. * feat: implement batch Push spilt * update go mod * feat: implement kafka producer and consumer. * update format, * add PushMQ log. * feat: update Handler logic. * update MQ logic. * update * update * fix: update OfflinePushConsumerHandler.
This commit is contained in:
parent
c581d43f17
commit
3381b85895
@ -14,12 +14,16 @@ toRedisTopic: toRedis
|
|||||||
toMongoTopic: toMongo
|
toMongoTopic: toMongo
|
||||||
# Kafka topic for push notifications
|
# Kafka topic for push notifications
|
||||||
toPushTopic: toPush
|
toPushTopic: toPush
|
||||||
|
# Kafka topic for offline push notifications
|
||||||
|
toOfflinePushTopic: toOfflinePush
|
||||||
# Consumer group ID for Redis topic
|
# Consumer group ID for Redis topic
|
||||||
toRedisGroupID: redis
|
toRedisGroupID: redis
|
||||||
# Consumer group ID for MongoDB topic
|
# Consumer group ID for MongoDB topic
|
||||||
toMongoGroupID: mongo
|
toMongoGroupID: mongo
|
||||||
# Consumer group ID for push notifications topic
|
# Consumer group ID for push notifications topic
|
||||||
toPushGroupID: push
|
toPushGroupID: push
|
||||||
|
# Consumer group ID for offline push notifications topic
|
||||||
|
toOfflinePushGroupID: offlinePush
|
||||||
# TLS (Transport Layer Security) configuration
|
# TLS (Transport Layer Security) configuration
|
||||||
tls:
|
tls:
|
||||||
# Enable or disable TLS
|
# Enable or disable TLS
|
||||||
|
2
go.mod
2
go.mod
@ -12,7 +12,7 @@ require (
|
|||||||
github.com/gorilla/websocket v1.5.1
|
github.com/gorilla/websocket v1.5.1
|
||||||
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0
|
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0
|
||||||
github.com/mitchellh/mapstructure v1.5.0
|
github.com/mitchellh/mapstructure v1.5.0
|
||||||
github.com/openimsdk/protocol v0.0.72-alpha.13
|
github.com/openimsdk/protocol v0.0.72-alpha.17
|
||||||
github.com/openimsdk/tools v0.0.50-alpha.11
|
github.com/openimsdk/tools v0.0.50-alpha.11
|
||||||
github.com/pkg/errors v0.9.1 // indirect
|
github.com/pkg/errors v0.9.1 // indirect
|
||||||
github.com/prometheus/client_golang v1.18.0
|
github.com/prometheus/client_golang v1.18.0
|
||||||
|
4
go.sum
4
go.sum
@ -319,8 +319,8 @@ github.com/onsi/gomega v1.25.0 h1:Vw7br2PCDYijJHSfBOWhov+8cAnUf8MfMaIOV323l6Y=
|
|||||||
github.com/onsi/gomega v1.25.0/go.mod h1:r+zV744Re+DiYCIPRlYOTxn0YkOLcAnW8k1xXdMPGhM=
|
github.com/onsi/gomega v1.25.0/go.mod h1:r+zV744Re+DiYCIPRlYOTxn0YkOLcAnW8k1xXdMPGhM=
|
||||||
github.com/openimsdk/gomake v0.0.14-alpha.5 h1:VY9c5x515lTfmdhhPjMvR3BBRrRquAUCFsz7t7vbv7Y=
|
github.com/openimsdk/gomake v0.0.14-alpha.5 h1:VY9c5x515lTfmdhhPjMvR3BBRrRquAUCFsz7t7vbv7Y=
|
||||||
github.com/openimsdk/gomake v0.0.14-alpha.5/go.mod h1:PndCozNc2IsQIciyn9mvEblYWZwJmAI+06z94EY+csI=
|
github.com/openimsdk/gomake v0.0.14-alpha.5/go.mod h1:PndCozNc2IsQIciyn9mvEblYWZwJmAI+06z94EY+csI=
|
||||||
github.com/openimsdk/protocol v0.0.72-alpha.13 h1:ILpvuxWGrVJMVCPRodOQcrSMFKUBzLahBPb8GkITWSc=
|
github.com/openimsdk/protocol v0.0.72-alpha.17 h1:kB7eyjJHdkc8lpSlLIHskHzbodxkIG4eaK908iQLVdI=
|
||||||
github.com/openimsdk/protocol v0.0.72-alpha.13/go.mod h1:OZQA9FR55lseYoN2Ql1XAHYKHJGu7OMNkUbuekrKCM8=
|
github.com/openimsdk/protocol v0.0.72-alpha.17/go.mod h1:OZQA9FR55lseYoN2Ql1XAHYKHJGu7OMNkUbuekrKCM8=
|
||||||
github.com/openimsdk/tools v0.0.50-alpha.11 h1:ClhkRjUVJWbmOiQ14G6do/ES1a6ZueDITv40Apwq/Tc=
|
github.com/openimsdk/tools v0.0.50-alpha.11 h1:ClhkRjUVJWbmOiQ14G6do/ES1a6ZueDITv40Apwq/Tc=
|
||||||
github.com/openimsdk/tools v0.0.50-alpha.11/go.mod h1:h1cYmfyaVtgFbKmb1Cfsl8XwUOMTt8ubVUQrdGtsUh4=
|
github.com/openimsdk/tools v0.0.50-alpha.11/go.mod h1:h1cYmfyaVtgFbKmb1Cfsl8XwUOMTt8ubVUQrdGtsUh4=
|
||||||
github.com/pelletier/go-toml/v2 v2.1.0 h1:FnwAJ4oYMvbT/34k9zzHuZNrhlz48GB3/s6at6/MHO4=
|
github.com/pelletier/go-toml/v2 v2.1.0 h1:FnwAJ4oYMvbT/34k9zzHuZNrhlz48GB3/s6at6/MHO4=
|
||||||
|
@ -111,6 +111,7 @@ func Start(ctx context.Context, index int, config *Config) error {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
msgTransfer := &MsgTransfer{
|
msgTransfer := &MsgTransfer{
|
||||||
historyCH: historyCH,
|
historyCH: historyCH,
|
||||||
historyMongoCH: historyMongoCH,
|
historyMongoCH: historyMongoCH,
|
||||||
|
@ -18,11 +18,12 @@ import (
|
|||||||
"context"
|
"context"
|
||||||
"crypto/sha256"
|
"crypto/sha256"
|
||||||
"encoding/hex"
|
"encoding/hex"
|
||||||
"github.com/openimsdk/open-im-server/v3/internal/push/offlinepush/options"
|
|
||||||
"strconv"
|
"strconv"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/openimsdk/open-im-server/v3/internal/push/offlinepush/options"
|
||||||
|
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/common/config"
|
"github.com/openimsdk/open-im-server/v3/pkg/common/config"
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/common/storage/cache"
|
"github.com/openimsdk/open-im-server/v3/pkg/common/storage/cache"
|
||||||
"github.com/openimsdk/tools/errs"
|
"github.com/openimsdk/tools/errs"
|
||||||
@ -91,6 +92,16 @@ func (g *Client) Push(ctx context.Context, userIDs []string, title, content stri
|
|||||||
for i, v := range s.GetSplitResult() {
|
for i, v := range s.GetSplitResult() {
|
||||||
go func(index int, userIDs []string) {
|
go func(index int, userIDs []string) {
|
||||||
defer wg.Done()
|
defer wg.Done()
|
||||||
|
for i := 0; i < len(userIDs); i += maxNum {
|
||||||
|
end := i + maxNum
|
||||||
|
if end > len(userIDs) {
|
||||||
|
end = len(userIDs)
|
||||||
|
}
|
||||||
|
if err = g.batchPush(ctx, token, userIDs[i:end], pushReq); err != nil {
|
||||||
|
log.ZError(ctx, "batchPush failed", err, "index", index, "token", token, "req", pushReq)
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
if err = g.batchPush(ctx, token, userIDs, pushReq); err != nil {
|
if err = g.batchPush(ctx, token, userIDs, pushReq); err != nil {
|
||||||
log.ZError(ctx, "batchPush failed", err, "index", index, "token", token, "req", pushReq)
|
log.ZError(ctx, "batchPush failed", err, "index", index, "token", token, "req", pushReq)
|
||||||
}
|
}
|
||||||
|
122
internal/push/offlinepush_handler.go
Normal file
122
internal/push/offlinepush_handler.go
Normal file
@ -0,0 +1,122 @@
|
|||||||
|
package push
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
|
||||||
|
"github.com/IBM/sarama"
|
||||||
|
"github.com/openimsdk/open-im-server/v3/internal/push/offlinepush"
|
||||||
|
"github.com/openimsdk/open-im-server/v3/internal/push/offlinepush/options"
|
||||||
|
"github.com/openimsdk/open-im-server/v3/pkg/common/prommetrics"
|
||||||
|
"github.com/openimsdk/protocol/constant"
|
||||||
|
pbpush "github.com/openimsdk/protocol/push"
|
||||||
|
"github.com/openimsdk/protocol/sdkws"
|
||||||
|
"github.com/openimsdk/tools/errs"
|
||||||
|
"github.com/openimsdk/tools/log"
|
||||||
|
"github.com/openimsdk/tools/mq/kafka"
|
||||||
|
"github.com/openimsdk/tools/utils/jsonutil"
|
||||||
|
"google.golang.org/protobuf/proto"
|
||||||
|
)
|
||||||
|
|
||||||
|
type OfflinePushConsumerHandler struct {
|
||||||
|
OfflinePushConsumerGroup *kafka.MConsumerGroup
|
||||||
|
offlinePusher offlinepush.OfflinePusher
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewOfflinePushConsumerHandler(config *Config, offlinePusher offlinepush.OfflinePusher) (*OfflinePushConsumerHandler, error) {
|
||||||
|
var offlinePushConsumerHandler OfflinePushConsumerHandler
|
||||||
|
var err error
|
||||||
|
offlinePushConsumerHandler.offlinePusher = offlinePusher
|
||||||
|
offlinePushConsumerHandler.OfflinePushConsumerGroup, err = kafka.NewMConsumerGroup(config.KafkaConfig.Build(), config.KafkaConfig.ToOfflineGroupID,
|
||||||
|
[]string{config.KafkaConfig.ToOfflinePushTopic}, true)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
return &offlinePushConsumerHandler, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (*OfflinePushConsumerHandler) Setup(sarama.ConsumerGroupSession) error { return nil }
|
||||||
|
func (*OfflinePushConsumerHandler) Cleanup(sarama.ConsumerGroupSession) error { return nil }
|
||||||
|
func (o *OfflinePushConsumerHandler) ConsumeClaim(sess sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error {
|
||||||
|
for msg := range claim.Messages() {
|
||||||
|
ctx := o.OfflinePushConsumerGroup.GetContextFromMsg(msg)
|
||||||
|
o.handleMsg2OfflinePush(ctx, msg.Value)
|
||||||
|
sess.MarkMessage(msg, "")
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (o *OfflinePushConsumerHandler) handleMsg2OfflinePush(ctx context.Context, msg []byte) {
|
||||||
|
offlinePushMsg := pbpush.PushMsgReq{}
|
||||||
|
if err := proto.Unmarshal(msg, &offlinePushMsg); err != nil {
|
||||||
|
log.ZError(ctx, "offline push Unmarshal msg err", err, "msg", string(msg))
|
||||||
|
return
|
||||||
|
}
|
||||||
|
if offlinePushMsg.MsgData == nil || offlinePushMsg.UserIDs == nil {
|
||||||
|
log.ZError(ctx, "offline push msg is empty", errs.New("offlinePushMsg is empty"), "userIDs", offlinePushMsg.UserIDs, "msg", offlinePushMsg.MsgData)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
log.ZInfo(ctx, "receive to OfflinePush MQ", "userIDs", offlinePushMsg.UserIDs, "msg", offlinePushMsg.MsgData)
|
||||||
|
|
||||||
|
err := o.offlinePushMsg(ctx, offlinePushMsg.MsgData, offlinePushMsg.UserIDs)
|
||||||
|
if err != nil {
|
||||||
|
log.ZWarn(ctx, "offline push failed", err, "msg", offlinePushMsg.String())
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *OfflinePushConsumerHandler) getOfflinePushInfos(msg *sdkws.MsgData) (title, content string, opts *options.Opts, err error) {
|
||||||
|
type AtTextElem struct {
|
||||||
|
Text string `json:"text,omitempty"`
|
||||||
|
AtUserList []string `json:"atUserList,omitempty"`
|
||||||
|
IsAtSelf bool `json:"isAtSelf"`
|
||||||
|
}
|
||||||
|
|
||||||
|
opts = &options.Opts{Signal: &options.Signal{}}
|
||||||
|
if msg.OfflinePushInfo != nil {
|
||||||
|
opts.IOSBadgeCount = msg.OfflinePushInfo.IOSBadgeCount
|
||||||
|
opts.IOSPushSound = msg.OfflinePushInfo.IOSPushSound
|
||||||
|
opts.Ex = msg.OfflinePushInfo.Ex
|
||||||
|
}
|
||||||
|
|
||||||
|
if msg.OfflinePushInfo != nil {
|
||||||
|
title = msg.OfflinePushInfo.Title
|
||||||
|
content = msg.OfflinePushInfo.Desc
|
||||||
|
}
|
||||||
|
if title == "" {
|
||||||
|
switch msg.ContentType {
|
||||||
|
case constant.Text:
|
||||||
|
fallthrough
|
||||||
|
case constant.Picture:
|
||||||
|
fallthrough
|
||||||
|
case constant.Voice:
|
||||||
|
fallthrough
|
||||||
|
case constant.Video:
|
||||||
|
fallthrough
|
||||||
|
case constant.File:
|
||||||
|
title = constant.ContentType2PushContent[int64(msg.ContentType)]
|
||||||
|
case constant.AtText:
|
||||||
|
ac := AtTextElem{}
|
||||||
|
_ = jsonutil.JsonStringToStruct(string(msg.Content), &ac)
|
||||||
|
case constant.SignalingNotification:
|
||||||
|
title = constant.ContentType2PushContent[constant.SignalMsg]
|
||||||
|
default:
|
||||||
|
title = constant.ContentType2PushContent[constant.Common]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if content == "" {
|
||||||
|
content = title
|
||||||
|
}
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *OfflinePushConsumerHandler) offlinePushMsg(ctx context.Context, msg *sdkws.MsgData, offlinePushUserIDs []string) error {
|
||||||
|
title, content, opts, err := c.getOfflinePushInfos(msg)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
err = c.offlinePusher.Push(ctx, offlinePushUserIDs, title, content, opts)
|
||||||
|
if err != nil {
|
||||||
|
prommetrics.MsgOfflinePushFailedCounter.Inc()
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
@ -2,6 +2,7 @@ package push
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
|
||||||
"github.com/openimsdk/open-im-server/v3/internal/push/offlinepush"
|
"github.com/openimsdk/open-im-server/v3/internal/push/offlinepush"
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/common/config"
|
"github.com/openimsdk/open-im-server/v3/pkg/common/config"
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/common/storage/cache/redis"
|
"github.com/openimsdk/open-im-server/v3/pkg/common/storage/cache/redis"
|
||||||
@ -17,12 +18,12 @@ type pushServer struct {
|
|||||||
disCov discovery.SvcDiscoveryRegistry
|
disCov discovery.SvcDiscoveryRegistry
|
||||||
offlinePusher offlinepush.OfflinePusher
|
offlinePusher offlinepush.OfflinePusher
|
||||||
pushCh *ConsumerHandler
|
pushCh *ConsumerHandler
|
||||||
|
offlinePushCh *OfflinePushConsumerHandler
|
||||||
}
|
}
|
||||||
|
|
||||||
type Config struct {
|
type Config struct {
|
||||||
RpcConfig config.Push
|
RpcConfig config.Push
|
||||||
RedisConfig config.Redis
|
RedisConfig config.Redis
|
||||||
MongodbConfig config.Mongo
|
|
||||||
KafkaConfig config.Kafka
|
KafkaConfig config.Kafka
|
||||||
NotificationConfig config.Notification
|
NotificationConfig config.Notification
|
||||||
Share config.Share
|
Share config.Share
|
||||||
@ -55,18 +56,30 @@ func Start(ctx context.Context, config *Config, client discovery.SvcDiscoveryReg
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
database := controller.NewPushDatabase(cacheModel)
|
|
||||||
|
|
||||||
consumer, err := NewConsumerHandler(config, offlinePusher, rdb, client)
|
database := controller.NewPushDatabase(cacheModel, &config.KafkaConfig)
|
||||||
|
|
||||||
|
consumer, err := NewConsumerHandler(config, database, offlinePusher, rdb, client)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
offlinePushConsumer, err := NewOfflinePushConsumerHandler(config, offlinePusher)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
pbpush.RegisterPushMsgServiceServer(server, &pushServer{
|
pbpush.RegisterPushMsgServiceServer(server, &pushServer{
|
||||||
database: database,
|
database: database,
|
||||||
disCov: client,
|
disCov: client,
|
||||||
offlinePusher: offlinePusher,
|
offlinePusher: offlinePusher,
|
||||||
pushCh: consumer,
|
pushCh: consumer,
|
||||||
|
offlinePushCh: offlinePushConsumer,
|
||||||
})
|
})
|
||||||
|
|
||||||
go consumer.pushConsumerGroup.RegisterHandleAndConsumer(ctx, consumer)
|
go consumer.pushConsumerGroup.RegisterHandleAndConsumer(ctx, consumer)
|
||||||
|
|
||||||
|
go offlinePushConsumer.OfflinePushConsumerGroup.RegisterHandleAndConsumer(ctx, offlinePushConsumer)
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -1,33 +1,20 @@
|
|||||||
// Copyright © 2023 OpenIM. All rights reserved.
|
|
||||||
//
|
|
||||||
// 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 push
|
package push
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
|
|
||||||
"github.com/IBM/sarama"
|
"github.com/IBM/sarama"
|
||||||
"github.com/openimsdk/open-im-server/v3/internal/push/offlinepush"
|
"github.com/openimsdk/open-im-server/v3/internal/push/offlinepush"
|
||||||
"github.com/openimsdk/open-im-server/v3/internal/push/offlinepush/options"
|
"github.com/openimsdk/open-im-server/v3/internal/push/offlinepush/options"
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/common/prommetrics"
|
"github.com/openimsdk/open-im-server/v3/pkg/common/prommetrics"
|
||||||
|
"github.com/openimsdk/open-im-server/v3/pkg/common/storage/controller"
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/common/webhook"
|
"github.com/openimsdk/open-im-server/v3/pkg/common/webhook"
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/msgprocessor"
|
"github.com/openimsdk/open-im-server/v3/pkg/msgprocessor"
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/rpccache"
|
"github.com/openimsdk/open-im-server/v3/pkg/rpccache"
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/rpcclient"
|
"github.com/openimsdk/open-im-server/v3/pkg/rpcclient"
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/util/conversationutil"
|
"github.com/openimsdk/open-im-server/v3/pkg/util/conversationutil"
|
||||||
"github.com/openimsdk/protocol/constant"
|
"github.com/openimsdk/protocol/constant"
|
||||||
pbchat "github.com/openimsdk/protocol/msg"
|
|
||||||
"github.com/openimsdk/protocol/msggateway"
|
"github.com/openimsdk/protocol/msggateway"
|
||||||
pbpush "github.com/openimsdk/protocol/push"
|
pbpush "github.com/openimsdk/protocol/push"
|
||||||
"github.com/openimsdk/protocol/sdkws"
|
"github.com/openimsdk/protocol/sdkws"
|
||||||
@ -46,6 +33,7 @@ type ConsumerHandler struct {
|
|||||||
pushConsumerGroup *kafka.MConsumerGroup
|
pushConsumerGroup *kafka.MConsumerGroup
|
||||||
offlinePusher offlinepush.OfflinePusher
|
offlinePusher offlinepush.OfflinePusher
|
||||||
onlinePusher OnlinePusher
|
onlinePusher OnlinePusher
|
||||||
|
pushDatabase controller.PushDatabase
|
||||||
onlineCache *rpccache.OnlineCache
|
onlineCache *rpccache.OnlineCache
|
||||||
groupLocalCache *rpccache.GroupLocalCache
|
groupLocalCache *rpccache.GroupLocalCache
|
||||||
conversationLocalCache *rpccache.ConversationLocalCache
|
conversationLocalCache *rpccache.ConversationLocalCache
|
||||||
@ -56,7 +44,7 @@ type ConsumerHandler struct {
|
|||||||
config *Config
|
config *Config
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewConsumerHandler(config *Config, offlinePusher offlinepush.OfflinePusher, rdb redis.UniversalClient,
|
func NewConsumerHandler(config *Config, database controller.PushDatabase, offlinePusher offlinepush.OfflinePusher, rdb redis.UniversalClient,
|
||||||
client discovery.SvcDiscoveryRegistry) (*ConsumerHandler, error) {
|
client discovery.SvcDiscoveryRegistry) (*ConsumerHandler, error) {
|
||||||
var consumerHandler ConsumerHandler
|
var consumerHandler ConsumerHandler
|
||||||
var err error
|
var err error
|
||||||
@ -65,6 +53,7 @@ func NewConsumerHandler(config *Config, offlinePusher offlinepush.OfflinePusher,
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
userRpcClient := rpcclient.NewUserRpcClient(client, config.Share.RpcRegisterName.User, config.Share.IMAdminUserID)
|
userRpcClient := rpcclient.NewUserRpcClient(client, config.Share.RpcRegisterName.User, config.Share.IMAdminUserID)
|
||||||
consumerHandler.offlinePusher = offlinePusher
|
consumerHandler.offlinePusher = offlinePusher
|
||||||
consumerHandler.onlinePusher = NewOnlinePusher(client, config)
|
consumerHandler.onlinePusher = NewOnlinePusher(client, config)
|
||||||
@ -75,43 +64,42 @@ func NewConsumerHandler(config *Config, offlinePusher offlinepush.OfflinePusher,
|
|||||||
consumerHandler.conversationLocalCache = rpccache.NewConversationLocalCache(consumerHandler.conversationRpcClient, &config.LocalCacheConfig, rdb)
|
consumerHandler.conversationLocalCache = rpccache.NewConversationLocalCache(consumerHandler.conversationRpcClient, &config.LocalCacheConfig, rdb)
|
||||||
consumerHandler.webhookClient = webhook.NewWebhookClient(config.WebhooksConfig.URL)
|
consumerHandler.webhookClient = webhook.NewWebhookClient(config.WebhooksConfig.URL)
|
||||||
consumerHandler.config = config
|
consumerHandler.config = config
|
||||||
|
consumerHandler.pushDatabase = database
|
||||||
consumerHandler.onlineCache = rpccache.NewOnlineCache(userRpcClient, consumerHandler.groupLocalCache, rdb, nil)
|
consumerHandler.onlineCache = rpccache.NewOnlineCache(userRpcClient, consumerHandler.groupLocalCache, rdb, nil)
|
||||||
return &consumerHandler, nil
|
return &consumerHandler, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *ConsumerHandler) handleMs2PsChat(ctx context.Context, msg []byte) {
|
func (c *ConsumerHandler) handleMs2PsChat(ctx context.Context, msg []byte) {
|
||||||
msgFromMQ := pbchat.PushMsgDataToMQ{}
|
msgFromMQ := pbpush.PushMsgReq{}
|
||||||
if err := proto.Unmarshal(msg, &msgFromMQ); err != nil {
|
if err := proto.Unmarshal(msg, &msgFromMQ); err != nil {
|
||||||
log.ZError(ctx, "push Unmarshal msg err", err, "msg", string(msg))
|
log.ZError(ctx, "push Unmarshal msg err", err, "msg", string(msg))
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
pbData := &pbpush.PushMsgReq{
|
|
||||||
MsgData: msgFromMQ.MsgData,
|
|
||||||
ConversationID: msgFromMQ.ConversationID,
|
|
||||||
}
|
|
||||||
sec := msgFromMQ.MsgData.SendTime / 1000
|
sec := msgFromMQ.MsgData.SendTime / 1000
|
||||||
nowSec := timeutil.GetCurrentTimestampBySecond()
|
nowSec := timeutil.GetCurrentTimestampBySecond()
|
||||||
|
|
||||||
if nowSec-sec > 10 {
|
if nowSec-sec > 10 {
|
||||||
prommetrics.MsgLoneTimePushCounter.Inc()
|
prommetrics.MsgLoneTimePushCounter.Inc()
|
||||||
log.ZWarn(ctx, "it’s been a while since the message was sent", nil, "msg", pbData.String(), "sec", sec, "nowSec", nowSec, "nowSec-sec", nowSec-sec)
|
log.ZWarn(ctx, "it’s been a while since the message was sent", nil, "msg", msgFromMQ.String(), "sec", sec, "nowSec", nowSec, "nowSec-sec", nowSec-sec)
|
||||||
}
|
}
|
||||||
var err error
|
var err error
|
||||||
|
|
||||||
switch msgFromMQ.MsgData.SessionType {
|
switch msgFromMQ.MsgData.SessionType {
|
||||||
case constant.ReadGroupChatType:
|
case constant.ReadGroupChatType:
|
||||||
err = c.Push2Group(ctx, pbData.MsgData.GroupID, pbData.MsgData)
|
err = c.Push2Group(ctx, msgFromMQ.MsgData.GroupID, msgFromMQ.MsgData)
|
||||||
default:
|
default:
|
||||||
var pushUserIDList []string
|
var pushUserIDList []string
|
||||||
isSenderSync := datautil.GetSwitchFromOptions(pbData.MsgData.Options, constant.IsSenderSync)
|
isSenderSync := datautil.GetSwitchFromOptions(msgFromMQ.MsgData.Options, constant.IsSenderSync)
|
||||||
if !isSenderSync || pbData.MsgData.SendID == pbData.MsgData.RecvID {
|
if !isSenderSync || msgFromMQ.MsgData.SendID == msgFromMQ.MsgData.RecvID {
|
||||||
pushUserIDList = append(pushUserIDList, pbData.MsgData.RecvID)
|
pushUserIDList = append(pushUserIDList, msgFromMQ.MsgData.RecvID)
|
||||||
} else {
|
} else {
|
||||||
pushUserIDList = append(pushUserIDList, pbData.MsgData.RecvID, pbData.MsgData.SendID)
|
pushUserIDList = append(pushUserIDList, msgFromMQ.MsgData.RecvID, msgFromMQ.MsgData.SendID)
|
||||||
}
|
}
|
||||||
err = c.Push2User(ctx, pushUserIDList, pbData.MsgData)
|
err = c.Push2User(ctx, pushUserIDList, msgFromMQ.MsgData)
|
||||||
}
|
}
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.ZWarn(ctx, "push failed", err, "msg", pbData.String())
|
log.ZWarn(ctx, "push failed", err, "msg", msgFromMQ.String())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -246,28 +234,34 @@ func (c *ConsumerHandler) Push2Group(ctx context.Context, groupID string, msg *s
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Use offline push messaging
|
// Use offline push messaging
|
||||||
if len(needOfflinePushUserIDs) > 0 {
|
if len(needOfflinePushUserIDs) > 0 {
|
||||||
var offlinePushUserIDs []string
|
c.asyncOfflinePush(ctx, needOfflinePushUserIDs, msg)
|
||||||
err = c.webhookBeforeOfflinePush(ctx, &c.config.WebhooksConfig.BeforeOfflinePush, needOfflinePushUserIDs, msg, &offlinePushUserIDs)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
if len(offlinePushUserIDs) > 0 {
|
|
||||||
needOfflinePushUserIDs = offlinePushUserIDs
|
|
||||||
}
|
|
||||||
|
|
||||||
err = c.offlinePushMsg(ctx, msg, needOfflinePushUserIDs)
|
|
||||||
if err != nil {
|
|
||||||
log.ZWarn(ctx, "offlinePushMsg failed", err, "groupID", groupID, "msg", msg)
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (c *ConsumerHandler) asyncOfflinePush(ctx context.Context, needOfflinePushUserIDs []string, msg *sdkws.MsgData) {
|
||||||
|
var offlinePushUserIDs []string
|
||||||
|
err := c.webhookBeforeOfflinePush(ctx, &c.config.WebhooksConfig.BeforeOfflinePush, needOfflinePushUserIDs, msg, &offlinePushUserIDs)
|
||||||
|
if err != nil {
|
||||||
|
log.ZWarn(ctx, "webhookBeforeOfflinePush failed", err, "msg", msg)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
if len(offlinePushUserIDs) > 0 {
|
||||||
|
needOfflinePushUserIDs = offlinePushUserIDs
|
||||||
|
}
|
||||||
|
if err := c.pushDatabase.MsgToOfflinePushMQ(ctx, conversationutil.GenConversationUniqueKeyForSingle(msg.SendID, msg.RecvID), needOfflinePushUserIDs, msg); err != nil {
|
||||||
|
log.ZError(ctx, "Msg To OfflinePush MQ error", err, "needOfflinePushUserIDs",
|
||||||
|
needOfflinePushUserIDs, "msg", msg)
|
||||||
|
prommetrics.SingleChatMsgProcessFailedCounter.Inc()
|
||||||
|
return
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func (c *ConsumerHandler) groupMessagesHandler(ctx context.Context, groupID string, pushToUserIDs *[]string, msg *sdkws.MsgData) (err error) {
|
func (c *ConsumerHandler) groupMessagesHandler(ctx context.Context, groupID string, pushToUserIDs *[]string, msg *sdkws.MsgData) (err error) {
|
||||||
if len(*pushToUserIDs) == 0 {
|
if len(*pushToUserIDs) == 0 {
|
||||||
*pushToUserIDs, err = c.groupLocalCache.GetGroupMemberIDs(ctx, groupID)
|
*pushToUserIDs, err = c.groupLocalCache.GetGroupMemberIDs(ctx, groupID)
|
||||||
|
@ -37,7 +37,6 @@ func NewPushRpcCmd() *PushRpcCmd {
|
|||||||
ret.configMap = map[string]any{
|
ret.configMap = map[string]any{
|
||||||
OpenIMPushCfgFileName: &pushConfig.RpcConfig,
|
OpenIMPushCfgFileName: &pushConfig.RpcConfig,
|
||||||
RedisConfigFileName: &pushConfig.RedisConfig,
|
RedisConfigFileName: &pushConfig.RedisConfig,
|
||||||
MongodbConfigFileName: &pushConfig.MongodbConfig,
|
|
||||||
KafkaConfigFileName: &pushConfig.KafkaConfig,
|
KafkaConfigFileName: &pushConfig.KafkaConfig,
|
||||||
ShareFileName: &pushConfig.Share,
|
ShareFileName: &pushConfig.Share,
|
||||||
NotificationFileName: &pushConfig.NotificationConfig,
|
NotificationFileName: &pushConfig.NotificationConfig,
|
||||||
|
@ -73,18 +73,21 @@ type Mongo struct {
|
|||||||
MaxRetry int `mapstructure:"maxRetry"`
|
MaxRetry int `mapstructure:"maxRetry"`
|
||||||
}
|
}
|
||||||
type Kafka struct {
|
type Kafka struct {
|
||||||
Username string `mapstructure:"username"`
|
Username string `mapstructure:"username"`
|
||||||
Password string `mapstructure:"password"`
|
Password string `mapstructure:"password"`
|
||||||
ProducerAck string `mapstructure:"producerAck"`
|
ProducerAck string `mapstructure:"producerAck"`
|
||||||
CompressType string `mapstructure:"compressType"`
|
CompressType string `mapstructure:"compressType"`
|
||||||
Address []string `mapstructure:"address"`
|
Address []string `mapstructure:"address"`
|
||||||
ToRedisTopic string `mapstructure:"toRedisTopic"`
|
ToRedisTopic string `mapstructure:"toRedisTopic"`
|
||||||
ToMongoTopic string `mapstructure:"toMongoTopic"`
|
ToMongoTopic string `mapstructure:"toMongoTopic"`
|
||||||
ToPushTopic string `mapstructure:"toPushTopic"`
|
ToPushTopic string `mapstructure:"toPushTopic"`
|
||||||
ToRedisGroupID string `mapstructure:"toRedisGroupID"`
|
ToOfflinePushTopic string `mapstructure:"toOfflinePushTopic"`
|
||||||
ToMongoGroupID string `mapstructure:"toMongoGroupID"`
|
ToRedisGroupID string `mapstructure:"toRedisGroupID"`
|
||||||
ToPushGroupID string `mapstructure:"toPushGroupID"`
|
ToMongoGroupID string `mapstructure:"toMongoGroupID"`
|
||||||
Tls TLSConfig `mapstructure:"tls"`
|
ToPushGroupID string `mapstructure:"toPushGroupID"`
|
||||||
|
ToOfflineGroupID string `mapstructure:"toOfflinePushGroupID"`
|
||||||
|
|
||||||
|
Tls TLSConfig `mapstructure:"tls"`
|
||||||
}
|
}
|
||||||
type TLSConfig struct {
|
type TLSConfig struct {
|
||||||
EnableTLS bool `mapstructure:"enableTLS"`
|
EnableTLS bool `mapstructure:"enableTLS"`
|
||||||
|
@ -17,21 +17,45 @@ package controller
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
|
||||||
|
"github.com/openimsdk/open-im-server/v3/pkg/common/config"
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/common/storage/cache"
|
"github.com/openimsdk/open-im-server/v3/pkg/common/storage/cache"
|
||||||
|
"github.com/openimsdk/protocol/push"
|
||||||
|
"github.com/openimsdk/protocol/sdkws"
|
||||||
|
"github.com/openimsdk/tools/log"
|
||||||
|
"github.com/openimsdk/tools/mq/kafka"
|
||||||
)
|
)
|
||||||
|
|
||||||
type PushDatabase interface {
|
type PushDatabase interface {
|
||||||
DelFcmToken(ctx context.Context, userID string, platformID int) error
|
DelFcmToken(ctx context.Context, userID string, platformID int) error
|
||||||
|
MsgToOfflinePushMQ(ctx context.Context, key string, userIDs []string, msg2mq *sdkws.MsgData) error
|
||||||
}
|
}
|
||||||
|
|
||||||
type pushDataBase struct {
|
type pushDataBase struct {
|
||||||
cache cache.ThirdCache
|
cache cache.ThirdCache
|
||||||
|
producerToOfflinePush *kafka.Producer
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewPushDatabase(cache cache.ThirdCache) PushDatabase {
|
func NewPushDatabase(cache cache.ThirdCache, kafkaConf *config.Kafka) PushDatabase {
|
||||||
return &pushDataBase{cache: cache}
|
conf, err := kafka.BuildProducerConfig(*kafkaConf.Build())
|
||||||
|
if err != nil {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
producerToOfflinePush, err := kafka.NewKafkaProducer(conf, kafkaConf.Address, kafkaConf.ToOfflinePushTopic)
|
||||||
|
if err != nil {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
return &pushDataBase{
|
||||||
|
cache: cache,
|
||||||
|
producerToOfflinePush: producerToOfflinePush,
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *pushDataBase) DelFcmToken(ctx context.Context, userID string, platformID int) error {
|
func (p *pushDataBase) DelFcmToken(ctx context.Context, userID string, platformID int) error {
|
||||||
return p.cache.DelFcmToken(ctx, userID, platformID)
|
return p.cache.DelFcmToken(ctx, userID, platformID)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (p *pushDataBase) MsgToOfflinePushMQ(ctx context.Context, key string, userIDs []string, msg2mq *sdkws.MsgData) error {
|
||||||
|
_, _, err := p.producerToOfflinePush.SendMessage(ctx, key, &push.PushMsgReq{MsgData: msg2mq, UserIDs: userIDs})
|
||||||
|
log.ZInfo(ctx, "message is push to offlinePush topic", "key", key, "userIDs", userIDs, "msg", msg2mq.String())
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
@ -35,7 +35,7 @@ done
|
|||||||
echo "Kafka is ready. Creating topics..."
|
echo "Kafka is ready. Creating topics..."
|
||||||
|
|
||||||
|
|
||||||
topics=("toRedis" "toMongo" "toPush")
|
topics=("toRedis" "toMongo" "toPush" "toOfflinePush")
|
||||||
partitions=8
|
partitions=8
|
||||||
replicationFactor=1
|
replicationFactor=1
|
||||||
|
|
||||||
|
@ -18,6 +18,12 @@ import (
|
|||||||
"context"
|
"context"
|
||||||
"flag"
|
"flag"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"io/ioutil"
|
||||||
|
"log"
|
||||||
|
"os"
|
||||||
|
"path/filepath"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/common/cmd"
|
"github.com/openimsdk/open-im-server/v3/pkg/common/cmd"
|
||||||
"github.com/openimsdk/open-im-server/v3/pkg/common/config"
|
"github.com/openimsdk/open-im-server/v3/pkg/common/config"
|
||||||
"github.com/openimsdk/tools/db/mongoutil"
|
"github.com/openimsdk/tools/db/mongoutil"
|
||||||
@ -27,11 +33,6 @@ import (
|
|||||||
"github.com/openimsdk/tools/mq/kafka"
|
"github.com/openimsdk/tools/mq/kafka"
|
||||||
"github.com/openimsdk/tools/s3/minio"
|
"github.com/openimsdk/tools/s3/minio"
|
||||||
"github.com/openimsdk/tools/system/program"
|
"github.com/openimsdk/tools/system/program"
|
||||||
"io/ioutil"
|
|
||||||
"log"
|
|
||||||
"os"
|
|
||||||
"path/filepath"
|
|
||||||
"time"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
const maxRetry = 180
|
const maxRetry = 180
|
||||||
@ -65,7 +66,7 @@ func CheckMinIO(ctx context.Context, config *config.Minio) error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func CheckKafka(ctx context.Context, conf *config.Kafka) error {
|
func CheckKafka(ctx context.Context, conf *config.Kafka) error {
|
||||||
return kafka.Check(ctx, conf.Build(), []string{conf.ToMongoTopic, conf.ToRedisTopic, conf.ToPushTopic})
|
return kafka.Check(ctx, conf.Build(), []string{conf.ToMongoTopic, conf.ToRedisTopic, conf.ToPushTopic, conf.ToOfflinePushTopic})
|
||||||
}
|
}
|
||||||
|
|
||||||
func initConfig(configDir string) (*config.Mongo, *config.Redis, *config.Kafka, *config.Minio, *config.Discovery, error) {
|
func initConfig(configDir string) (*config.Mongo, *config.Redis, *config.Kafka, *config.Minio, *config.Discovery, error) {
|
||||||
|
Loading…
x
Reference in New Issue
Block a user