Accumulated changes for message queue (#6600)
* rename * set agent address * refactor * add agent sub * pub messages * grpc new client * can publish records via agent * send init message with session id * fmt * check cancelled request while waiting * use sessionId * handle possible nil stream * subscriber process messages * separate debug port * use atomic int64 * less logs * minor * skip io.EOF * rename * remove unused * use saved offsets * do not reuse session, since always session id is new after restart remove last active ts from SessionEntry * simplify printing * purge unused * just proxy the subscription, skipping the session step * adjust offset types * subscribe offset type and possible value * start after the known tsns * avoid wrongly set startPosition * move * remove * refactor * typo * fix * fix changed path
This commit is contained in:
@@ -1,14 +0,0 @@
|
||||
package agent_client
|
||||
|
||||
import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
)
|
||||
|
||||
func (a *PublishSession) PublishMessageRecord(key []byte, record *schema_pb.RecordValue) error {
|
||||
return a.stream.Send(&mq_agent_pb.PublishRecordRequest{
|
||||
SessionId: a.sessionId,
|
||||
Key: key,
|
||||
Value: record,
|
||||
})
|
||||
}
|
||||
@@ -1,17 +0,0 @@
|
||||
package agent_client
|
||||
|
||||
import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
)
|
||||
|
||||
func (a *SubscribeSession) SubscribeMessageRecord(
|
||||
onEachMessageFn func(key []byte, record *schema_pb.RecordValue),
|
||||
onCompletionFn func()) error {
|
||||
for {
|
||||
resp, err := a.stream.Recv()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
onEachMessageFn(resp.Key, resp.Value)
|
||||
}
|
||||
}
|
||||
@@ -4,10 +4,10 @@ import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/schema"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
)
|
||||
|
||||
type PublishSession struct {
|
||||
@@ -15,13 +15,12 @@ type PublishSession struct {
|
||||
partitionCount int
|
||||
publisherName string
|
||||
stream grpc.BidiStreamingClient[mq_agent_pb.PublishRecordRequest, mq_agent_pb.PublishRecordResponse]
|
||||
sessionId int64
|
||||
}
|
||||
|
||||
func NewPublishSession(agentAddress string, topicSchema *schema.Schema, partitionCount int, publisherName string) (*PublishSession, error) {
|
||||
|
||||
// call local agent grpc server to create a new session
|
||||
clientConn, err := pb.GrpcDial(context.Background(), agentAddress, true, grpc.WithInsecure())
|
||||
clientConn, err := grpc.NewClient(agentAddress, grpc.WithTransportCredentials(insecure.NewCredentials()))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("dial agent server %s: %v", agentAddress, err)
|
||||
}
|
||||
@@ -48,12 +47,17 @@ func NewPublishSession(agentAddress string, topicSchema *schema.Schema, partitio
|
||||
return nil, fmt.Errorf("publish record: %v", err)
|
||||
}
|
||||
|
||||
if err = stream.Send(&mq_agent_pb.PublishRecordRequest{
|
||||
SessionId: resp.SessionId,
|
||||
}); err != nil {
|
||||
return nil, fmt.Errorf("send session id: %v", err)
|
||||
}
|
||||
|
||||
return &PublishSession{
|
||||
schema: topicSchema,
|
||||
partitionCount: partitionCount,
|
||||
publisherName: publisherName,
|
||||
stream: stream,
|
||||
sessionId: resp.SessionId,
|
||||
}, nil
|
||||
}
|
||||
|
||||
@@ -68,3 +72,10 @@ func (a *PublishSession) CloseSession() error {
|
||||
a.schema = nil
|
||||
return err
|
||||
}
|
||||
|
||||
func (a *PublishSession) PublishMessageRecord(key []byte, record *schema_pb.RecordValue) error {
|
||||
return a.stream.Send(&mq_agent_pb.PublishRecordRequest{
|
||||
Key: key,
|
||||
Value: record,
|
||||
})
|
||||
}
|
||||
|
||||
@@ -4,50 +4,48 @@ import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
)
|
||||
|
||||
type SubscribeOption struct {
|
||||
ConsumerGroup string
|
||||
ConsumerGroupInstanceId string
|
||||
Topic topic.Topic
|
||||
OffsetType schema_pb.OffsetType
|
||||
OffsetTsNs int64
|
||||
Filter string
|
||||
MaxSubscribedPartitions int32
|
||||
PerPartitionConcurrency int32
|
||||
SlidingWindowSize int32
|
||||
}
|
||||
|
||||
type SubscribeSession struct {
|
||||
Option *SubscribeOption
|
||||
stream grpc.BidiStreamingClient[mq_agent_pb.SubscribeRecordRequest, mq_agent_pb.SubscribeRecordResponse]
|
||||
sessionId int64
|
||||
Option *SubscribeOption
|
||||
stream grpc.BidiStreamingClient[mq_agent_pb.SubscribeRecordRequest, mq_agent_pb.SubscribeRecordResponse]
|
||||
}
|
||||
|
||||
func NewSubscribeSession(agentAddress string, option *SubscribeOption) (*SubscribeSession, error) {
|
||||
// call local agent grpc server to create a new session
|
||||
clientConn, err := pb.GrpcDial(context.Background(), agentAddress, true, grpc.WithInsecure())
|
||||
clientConn, err := grpc.NewClient(agentAddress, grpc.WithTransportCredentials(insecure.NewCredentials()))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("dial agent server %s: %v", agentAddress, err)
|
||||
}
|
||||
agentClient := mq_agent_pb.NewSeaweedMessagingAgentClient(clientConn)
|
||||
|
||||
resp, err := agentClient.StartSubscribeSession(context.Background(), &mq_agent_pb.StartSubscribeSessionRequest{
|
||||
initRequest := &mq_agent_pb.SubscribeRecordRequest_InitSubscribeRecordRequest{
|
||||
ConsumerGroup: option.ConsumerGroup,
|
||||
ConsumerGroupInstanceId: option.ConsumerGroupInstanceId,
|
||||
Topic: &schema_pb.Topic{
|
||||
Namespace: option.Topic.Namespace,
|
||||
Name: option.Topic.Name,
|
||||
},
|
||||
OffsetType: option.OffsetType,
|
||||
OffsetTsNs: option.OffsetTsNs,
|
||||
MaxSubscribedPartitions: option.MaxSubscribedPartitions,
|
||||
Filter: option.Filter,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if resp.Error != "" {
|
||||
return nil, fmt.Errorf("start subscribe session: %v", resp.Error)
|
||||
SlidingWindowSize: option.SlidingWindowSize,
|
||||
}
|
||||
|
||||
stream, err := agentClient.SubscribeRecord(context.Background())
|
||||
@@ -55,9 +53,35 @@ func NewSubscribeSession(agentAddress string, option *SubscribeOption) (*Subscri
|
||||
return nil, fmt.Errorf("subscribe record: %v", err)
|
||||
}
|
||||
|
||||
if err = stream.Send(&mq_agent_pb.SubscribeRecordRequest{
|
||||
Init: initRequest,
|
||||
}); err != nil {
|
||||
return nil, fmt.Errorf("send session id: %v", err)
|
||||
}
|
||||
|
||||
return &SubscribeSession{
|
||||
Option: option,
|
||||
stream: stream,
|
||||
sessionId: resp.SessionId,
|
||||
Option: option,
|
||||
stream: stream,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *SubscribeSession) CloseSession() error {
|
||||
err := s.stream.CloseSend()
|
||||
return err
|
||||
}
|
||||
|
||||
func (a *SubscribeSession) SubscribeMessageRecord(
|
||||
onEachMessageFn func(key []byte, record *schema_pb.RecordValue),
|
||||
onCompletionFn func()) error {
|
||||
for {
|
||||
resp, err := a.stream.Recv()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
onEachMessageFn(resp.Key, resp.Value)
|
||||
}
|
||||
if onCompletionFn != nil {
|
||||
onCompletionFn()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
@@ -1,128 +0,0 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/agent_client"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/schema"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"log"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
)
|
||||
|
||||
var (
|
||||
messageCount = flag.Int("n", 1000, "message count")
|
||||
messageDelay = flag.Duration("d", time.Second, "delay between messages")
|
||||
concurrency = flag.Int("c", 4, "concurrent publishers")
|
||||
partitionCount = flag.Int("p", 6, "partition count")
|
||||
|
||||
clientName = flag.String("client", "c1", "client name")
|
||||
|
||||
namespace = flag.String("ns", "test", "namespace")
|
||||
t = flag.String("t", "test", "t")
|
||||
seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
|
||||
|
||||
counter int32
|
||||
)
|
||||
|
||||
func doPublish(publisher *agent_client.PublishSession, id int) {
|
||||
startTime := time.Now()
|
||||
for {
|
||||
i := atomic.AddInt32(&counter, 1)
|
||||
if i > int32(*messageCount) {
|
||||
break
|
||||
}
|
||||
// Simulate publishing a message
|
||||
myRecord := genMyRecord(int32(i))
|
||||
if err := publisher.PublishMessageRecord(myRecord.Key, myRecord.ToRecordValue()); err != nil {
|
||||
fmt.Println(err)
|
||||
break
|
||||
}
|
||||
if *messageDelay > 0 {
|
||||
time.Sleep(*messageDelay)
|
||||
fmt.Printf("sent %+v\n", string(myRecord.Key))
|
||||
}
|
||||
}
|
||||
elapsed := time.Since(startTime)
|
||||
log.Printf("Publisher %s-%d finished in %s", *clientName, id, elapsed)
|
||||
}
|
||||
|
||||
type MyRecord struct {
|
||||
Key []byte
|
||||
Field1 []byte
|
||||
Field2 string
|
||||
Field3 int32
|
||||
Field4 int64
|
||||
Field5 float32
|
||||
Field6 float64
|
||||
Field7 bool
|
||||
}
|
||||
|
||||
func genMyRecord(id int32) *MyRecord {
|
||||
return &MyRecord{
|
||||
Key: []byte(fmt.Sprintf("key-%s-%d", *clientName, id)),
|
||||
Field1: []byte(fmt.Sprintf("field1-%s-%d", *clientName, id)),
|
||||
Field2: fmt.Sprintf("field2-%s-%d", *clientName, id),
|
||||
Field3: id,
|
||||
Field4: int64(id),
|
||||
Field5: float32(id),
|
||||
Field6: float64(id),
|
||||
Field7: id%2 == 0,
|
||||
}
|
||||
}
|
||||
|
||||
func (r *MyRecord) ToRecordValue() *schema_pb.RecordValue {
|
||||
return schema.RecordBegin().
|
||||
SetBytes("key", r.Key).
|
||||
SetBytes("field1", r.Field1).
|
||||
SetString("field2", r.Field2).
|
||||
SetInt32("field3", r.Field3).
|
||||
SetInt64("field4", r.Field4).
|
||||
SetFloat("field5", r.Field5).
|
||||
SetDouble("field6", r.Field6).
|
||||
SetBool("field7", r.Field7).
|
||||
RecordEnd()
|
||||
}
|
||||
|
||||
func main() {
|
||||
flag.Parse()
|
||||
|
||||
recordType := schema.RecordTypeBegin().
|
||||
WithField("key", schema.TypeBytes).
|
||||
WithField("field1", schema.TypeBytes).
|
||||
WithField("field2", schema.TypeString).
|
||||
WithField("field3", schema.TypeInt32).
|
||||
WithField("field4", schema.TypeInt64).
|
||||
WithField("field5", schema.TypeFloat).
|
||||
WithField("field6", schema.TypeDouble).
|
||||
WithField("field7", schema.TypeBoolean).
|
||||
RecordTypeEnd()
|
||||
|
||||
session, err := agent_client.NewPublishSession("localhost:16777", schema.NewSchema(*namespace, *t, recordType), *partitionCount, *clientName)
|
||||
if err != nil {
|
||||
log.Printf("failed to create session: %v", err)
|
||||
return
|
||||
}
|
||||
defer session.CloseSession()
|
||||
|
||||
startTime := time.Now()
|
||||
|
||||
var wg sync.WaitGroup
|
||||
// Start multiple publishers
|
||||
for i := 0; i < *concurrency; i++ {
|
||||
wg.Add(1)
|
||||
go func(id int) {
|
||||
defer wg.Done()
|
||||
doPublish(session, id)
|
||||
}(i)
|
||||
}
|
||||
|
||||
// Wait for all publishers to finish
|
||||
wg.Wait()
|
||||
elapsed := time.Since(startTime)
|
||||
|
||||
log.Printf("Published %d messages in %s (%.2f msg/s)", *messageCount, elapsed, float64(*messageCount)/elapsed.Seconds())
|
||||
|
||||
}
|
||||
@@ -1,78 +0,0 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
|
||||
"log"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
|
||||
var (
|
||||
messageCount = flag.Int("n", 1000, "message count")
|
||||
concurrency = flag.Int("c", 4, "concurrent publishers")
|
||||
partitionCount = flag.Int("p", 6, "partition count")
|
||||
|
||||
clientName = flag.String("client", "c1", "client name")
|
||||
|
||||
namespace = flag.String("ns", "test", "namespace")
|
||||
t = flag.String("t", "test", "t")
|
||||
seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
|
||||
)
|
||||
|
||||
func doPublish(publisher *pub_client.TopicPublisher, id int) {
|
||||
startTime := time.Now()
|
||||
for i := 0; i < *messageCount / *concurrency; i++ {
|
||||
// Simulate publishing a message
|
||||
key := []byte(fmt.Sprintf("key-%s-%d-%d", *clientName, id, i))
|
||||
value := []byte(fmt.Sprintf("value-%s-%d-%d", *clientName, id, i))
|
||||
if err := publisher.Publish(key, value); err != nil {
|
||||
fmt.Println(err)
|
||||
break
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
// println("Published", string(key), string(value))
|
||||
}
|
||||
if err := publisher.FinishPublish(); err != nil {
|
||||
fmt.Println(err)
|
||||
}
|
||||
elapsed := time.Since(startTime)
|
||||
log.Printf("Publisher %s-%d finished in %s", *clientName, id, elapsed)
|
||||
}
|
||||
|
||||
func main() {
|
||||
flag.Parse()
|
||||
util_http.InitGlobalHttpClient()
|
||||
|
||||
config := &pub_client.PublisherConfiguration{
|
||||
Topic: topic.NewTopic(*namespace, *t),
|
||||
PartitionCount: int32(*partitionCount),
|
||||
Brokers: strings.Split(*seedBrokers, ","),
|
||||
PublisherName: *clientName,
|
||||
}
|
||||
publisher := pub_client.NewTopicPublisher(config)
|
||||
|
||||
startTime := time.Now()
|
||||
|
||||
var wg sync.WaitGroup
|
||||
// Start multiple publishers
|
||||
for i := 0; i < *concurrency; i++ {
|
||||
wg.Add(1)
|
||||
go func(id int) {
|
||||
defer wg.Done()
|
||||
doPublish(publisher, id)
|
||||
}(i)
|
||||
}
|
||||
|
||||
// Wait for all publishers to finish
|
||||
wg.Wait()
|
||||
elapsed := time.Since(startTime)
|
||||
publisher.Shutdown()
|
||||
|
||||
log.Printf("Published %d messages in %s (%.2f msg/s)", *messageCount, elapsed, float64(*messageCount)/elapsed.Seconds())
|
||||
|
||||
}
|
||||
@@ -1,138 +0,0 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/schema"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
|
||||
"log"
|
||||
"strings"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
)
|
||||
|
||||
var (
|
||||
messageCount = flag.Int("n", 1000, "message count")
|
||||
messageDelay = flag.Duration("d", time.Second, "delay between messages")
|
||||
concurrency = flag.Int("c", 4, "concurrent publishers")
|
||||
partitionCount = flag.Int("p", 6, "partition count")
|
||||
|
||||
clientName = flag.String("client", "c1", "client name")
|
||||
|
||||
namespace = flag.String("ns", "test", "namespace")
|
||||
t = flag.String("t", "test", "t")
|
||||
seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
|
||||
|
||||
counter int32
|
||||
)
|
||||
|
||||
func doPublish(publisher *pub_client.TopicPublisher, id int) {
|
||||
startTime := time.Now()
|
||||
for {
|
||||
i := atomic.AddInt32(&counter, 1)
|
||||
if i > int32(*messageCount) {
|
||||
break
|
||||
}
|
||||
// Simulate publishing a message
|
||||
myRecord := genMyRecord(int32(i))
|
||||
if err := publisher.PublishRecord(myRecord.Key, myRecord.ToRecordValue()); err != nil {
|
||||
fmt.Println(err)
|
||||
break
|
||||
}
|
||||
if *messageDelay > 0 {
|
||||
time.Sleep(*messageDelay)
|
||||
fmt.Printf("sent %+v\n", string(myRecord.Key))
|
||||
}
|
||||
}
|
||||
if err := publisher.FinishPublish(); err != nil {
|
||||
fmt.Println(err)
|
||||
}
|
||||
elapsed := time.Since(startTime)
|
||||
log.Printf("Publisher %s-%d finished in %s", *clientName, id, elapsed)
|
||||
}
|
||||
|
||||
type MyRecord struct {
|
||||
Key []byte
|
||||
Field1 []byte
|
||||
Field2 string
|
||||
Field3 int32
|
||||
Field4 int64
|
||||
Field5 float32
|
||||
Field6 float64
|
||||
Field7 bool
|
||||
}
|
||||
|
||||
func genMyRecord(id int32) *MyRecord {
|
||||
return &MyRecord{
|
||||
Key: []byte(fmt.Sprintf("key-%s-%d", *clientName, id)),
|
||||
Field1: []byte(fmt.Sprintf("field1-%s-%d", *clientName, id)),
|
||||
Field2: fmt.Sprintf("field2-%s-%d", *clientName, id),
|
||||
Field3: id,
|
||||
Field4: int64(id),
|
||||
Field5: float32(id),
|
||||
Field6: float64(id),
|
||||
Field7: id%2 == 0,
|
||||
}
|
||||
}
|
||||
|
||||
func (r *MyRecord) ToRecordValue() *schema_pb.RecordValue {
|
||||
return schema.RecordBegin().
|
||||
SetBytes("key", r.Key).
|
||||
SetBytes("field1", r.Field1).
|
||||
SetString("field2", r.Field2).
|
||||
SetInt32("field3", r.Field3).
|
||||
SetInt64("field4", r.Field4).
|
||||
SetFloat("field5", r.Field5).
|
||||
SetDouble("field6", r.Field6).
|
||||
SetBool("field7", r.Field7).
|
||||
RecordEnd()
|
||||
}
|
||||
|
||||
func main() {
|
||||
flag.Parse()
|
||||
util_http.InitGlobalHttpClient()
|
||||
|
||||
recordType := schema.RecordTypeBegin().
|
||||
WithField("key", schema.TypeBytes).
|
||||
WithField("field1", schema.TypeBytes).
|
||||
WithField("field2", schema.TypeString).
|
||||
WithField("field3", schema.TypeInt32).
|
||||
WithField("field4", schema.TypeInt64).
|
||||
WithField("field5", schema.TypeFloat).
|
||||
WithField("field6", schema.TypeDouble).
|
||||
WithField("field7", schema.TypeBoolean).
|
||||
RecordTypeEnd()
|
||||
|
||||
config := &pub_client.PublisherConfiguration{
|
||||
Topic: topic.NewTopic(*namespace, *t),
|
||||
PartitionCount: int32(*partitionCount),
|
||||
Brokers: strings.Split(*seedBrokers, ","),
|
||||
PublisherName: *clientName,
|
||||
RecordType: recordType,
|
||||
}
|
||||
publisher := pub_client.NewTopicPublisher(config)
|
||||
|
||||
startTime := time.Now()
|
||||
|
||||
var wg sync.WaitGroup
|
||||
// Start multiple publishers
|
||||
for i := 0; i < *concurrency; i++ {
|
||||
wg.Add(1)
|
||||
go func(id int) {
|
||||
defer wg.Done()
|
||||
doPublish(publisher, id)
|
||||
}(i)
|
||||
}
|
||||
|
||||
// Wait for all publishers to finish
|
||||
wg.Wait()
|
||||
elapsed := time.Since(startTime)
|
||||
publisher.Shutdown()
|
||||
|
||||
log.Printf("Published %d messages in %s (%.2f msg/s)", *messageCount, elapsed, float64(*messageCount)/elapsed.Seconds())
|
||||
|
||||
}
|
||||
@@ -1,64 +0,0 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"strings"
|
||||
)
|
||||
|
||||
var (
|
||||
namespace = flag.String("ns", "test", "namespace")
|
||||
t = flag.String("topic", "test", "topic")
|
||||
seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
|
||||
maxPartitionCount = flag.Int("maxPartitionCount", 3, "max partition count")
|
||||
perPartitionConcurrency = flag.Int("perPartitionConcurrency", 1, "per partition concurrency")
|
||||
|
||||
clientId = flag.Uint("client_id", uint(util.RandomInt32()), "client id")
|
||||
)
|
||||
|
||||
func main() {
|
||||
flag.Parse()
|
||||
util_http.InitGlobalHttpClient()
|
||||
|
||||
subscriberConfig := &sub_client.SubscriberConfiguration{
|
||||
ConsumerGroup: "test",
|
||||
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
|
||||
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
MaxPartitionCount: int32(*maxPartitionCount),
|
||||
SlidingWindowSize: int32(*perPartitionConcurrency),
|
||||
}
|
||||
|
||||
contentConfig := &sub_client.ContentConfiguration{
|
||||
Topic: topic.NewTopic(*namespace, *t),
|
||||
Filter: "",
|
||||
}
|
||||
|
||||
brokers := strings.Split(*seedBrokers, ",")
|
||||
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, make(chan sub_client.KeyedOffset, 1024))
|
||||
|
||||
counter := 0
|
||||
executors := util.NewLimitedConcurrentExecutor(int(subscriberConfig.SlidingWindowSize))
|
||||
subscriber.SetOnDataMessageFn(func(m *mq_pb.SubscribeMessageResponse_Data) {
|
||||
executors.Execute(func() {
|
||||
counter++
|
||||
println(string(m.Data.Key), "=>", string(m.Data.Value), counter)
|
||||
})
|
||||
})
|
||||
|
||||
subscriber.SetCompletionFunc(func() {
|
||||
glog.V(0).Infof("done received %d messages", counter)
|
||||
})
|
||||
|
||||
if err := subscriber.Subscribe(); err != nil {
|
||||
fmt.Println(err)
|
||||
}
|
||||
|
||||
}
|
||||
@@ -1,99 +0,0 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/protobuf/proto"
|
||||
"strings"
|
||||
"time"
|
||||
)
|
||||
|
||||
var (
|
||||
namespace = flag.String("ns", "test", "namespace")
|
||||
t = flag.String("topic", "test", "topic")
|
||||
seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
|
||||
maxPartitionCount = flag.Int("maxPartitionCount", 3, "max partition count")
|
||||
perPartitionConcurrency = flag.Int("perPartitionConcurrency", 1, "per partition concurrency")
|
||||
timeAgo = flag.Duration("timeAgo", 1*time.Hour, "start time before now. \"300ms\", \"1.5h\" or \"2h45m\". Valid time units are \"ns\", \"us\" (or \"µs\"), \"ms\", \"s\", \"m\", \"h\"")
|
||||
|
||||
clientId = flag.Uint("client_id", uint(util.RandomInt32()), "client id")
|
||||
)
|
||||
|
||||
type MyRecord struct {
|
||||
Key []byte
|
||||
Field1 []byte
|
||||
Field2 string
|
||||
Field3 int32
|
||||
Field4 int64
|
||||
Field5 float32
|
||||
Field6 float64
|
||||
Field7 bool
|
||||
}
|
||||
|
||||
func FromSchemaRecordValue(recordValue *schema_pb.RecordValue) *MyRecord {
|
||||
return &MyRecord{
|
||||
Key: recordValue.Fields["key"].GetBytesValue(),
|
||||
Field1: recordValue.Fields["field1"].GetBytesValue(),
|
||||
Field2: recordValue.Fields["field2"].GetStringValue(),
|
||||
Field3: recordValue.Fields["field3"].GetInt32Value(),
|
||||
Field4: recordValue.Fields["field4"].GetInt64Value(),
|
||||
Field5: recordValue.Fields["field5"].GetFloatValue(),
|
||||
Field6: recordValue.Fields["field6"].GetDoubleValue(),
|
||||
Field7: recordValue.Fields["field7"].GetBoolValue(),
|
||||
}
|
||||
}
|
||||
|
||||
func main() {
|
||||
flag.Parse()
|
||||
util_http.InitGlobalHttpClient()
|
||||
|
||||
subscriberConfig := &sub_client.SubscriberConfiguration{
|
||||
ConsumerGroup: "test",
|
||||
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
|
||||
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
MaxPartitionCount: int32(*maxPartitionCount),
|
||||
SlidingWindowSize: int32(*perPartitionConcurrency),
|
||||
}
|
||||
|
||||
contentConfig := &sub_client.ContentConfiguration{
|
||||
Topic: topic.NewTopic(*namespace, *t),
|
||||
Filter: "",
|
||||
// StartTime: time.Now().Add(-*timeAgo),
|
||||
}
|
||||
|
||||
brokers := strings.Split(*seedBrokers, ",")
|
||||
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, make(chan sub_client.KeyedOffset, 1024))
|
||||
|
||||
counter := 0
|
||||
executors := util.NewLimitedConcurrentExecutor(int(subscriberConfig.SlidingWindowSize))
|
||||
subscriber.SetOnDataMessageFn(func(m *mq_pb.SubscribeMessageResponse_Data) {
|
||||
executors.Execute(func() {
|
||||
counter++
|
||||
record := &schema_pb.RecordValue{}
|
||||
err := proto.Unmarshal(m.Data.Value, record)
|
||||
if err != nil {
|
||||
fmt.Printf("unmarshal record value: %v\n", err)
|
||||
} else {
|
||||
fmt.Printf("%s %d: %v\n", string(m.Data.Key), len(m.Data.Value), record)
|
||||
}
|
||||
})
|
||||
})
|
||||
|
||||
subscriber.SetCompletionFunc(func() {
|
||||
glog.V(0).Infof("done received %d messages", counter)
|
||||
})
|
||||
|
||||
if err := subscriber.Subscribe(); err != nil {
|
||||
fmt.Println(err)
|
||||
}
|
||||
|
||||
}
|
||||
@@ -34,8 +34,8 @@ type TopicPublisher struct {
|
||||
jobs []*EachPartitionPublishJob
|
||||
}
|
||||
|
||||
func NewTopicPublisher(config *PublisherConfiguration) *TopicPublisher {
|
||||
tp := &TopicPublisher{
|
||||
func NewTopicPublisher(config *PublisherConfiguration) (tp *TopicPublisher, err error) {
|
||||
tp = &TopicPublisher{
|
||||
partition2Buffer: interval.NewSearchTree[*buffered_queue.BufferedQueue[*mq_pb.DataMessage]](func(a, b int32) int {
|
||||
return int(a - b)
|
||||
}),
|
||||
@@ -46,7 +46,7 @@ func NewTopicPublisher(config *PublisherConfiguration) *TopicPublisher {
|
||||
wg := sync.WaitGroup{}
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
if err := tp.startSchedulerThread(&wg); err != nil {
|
||||
if err = tp.startSchedulerThread(&wg); err != nil {
|
||||
log.Println(err)
|
||||
return
|
||||
}
|
||||
@@ -54,7 +54,7 @@ func NewTopicPublisher(config *PublisherConfiguration) *TopicPublisher {
|
||||
|
||||
wg.Wait()
|
||||
|
||||
return tp
|
||||
return
|
||||
}
|
||||
|
||||
func (p *TopicPublisher) Shutdown() error {
|
||||
|
||||
@@ -7,7 +7,9 @@ import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util/buffered_queue"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/status"
|
||||
"log"
|
||||
"sort"
|
||||
@@ -33,6 +35,7 @@ type EachPartitionPublishJob struct {
|
||||
func (p *TopicPublisher) startSchedulerThread(wg *sync.WaitGroup) error {
|
||||
|
||||
if err := p.doConfigureTopic(); err != nil {
|
||||
wg.Done()
|
||||
return fmt.Errorf("configure topic %s: %v", p.config.Topic, err)
|
||||
}
|
||||
|
||||
@@ -111,6 +114,7 @@ func (p *TopicPublisher) onEachAssignments(generation int, assignments []*mq_pb.
|
||||
go func(job *EachPartitionPublishJob) {
|
||||
defer job.wg.Done()
|
||||
if err := p.doPublishToPartition(job); err != nil {
|
||||
log.Printf("publish to %s partition %v: %v", p.config.Topic, job.Partition, err)
|
||||
errChan <- EachPartitionError{assignment, err, generation}
|
||||
}
|
||||
}(job)
|
||||
@@ -126,7 +130,7 @@ func (p *TopicPublisher) doPublishToPartition(job *EachPartitionPublishJob) erro
|
||||
|
||||
log.Printf("connecting to %v for topic partition %+v", job.LeaderBroker, job.Partition)
|
||||
|
||||
grpcConnection, err := pb.GrpcDial(context.Background(), job.LeaderBroker, true, p.grpcDialOption)
|
||||
grpcConnection, err := grpc.NewClient(job.LeaderBroker, grpc.WithTransportCredentials(insecure.NewCredentials()), p.grpcDialOption)
|
||||
if err != nil {
|
||||
return fmt.Errorf("dial broker %s: %v", job.LeaderBroker, err)
|
||||
}
|
||||
@@ -225,7 +229,7 @@ func (p *TopicPublisher) doPublishToPartition(job *EachPartitionPublishJob) erro
|
||||
|
||||
func (p *TopicPublisher) doConfigureTopic() (err error) {
|
||||
if len(p.config.Brokers) == 0 {
|
||||
return fmt.Errorf("no bootstrap brokers")
|
||||
return fmt.Errorf("topic configuring found no bootstrap brokers")
|
||||
}
|
||||
var lastErr error
|
||||
for _, brokerAddress := range p.config.Brokers {
|
||||
@@ -256,7 +260,7 @@ func (p *TopicPublisher) doConfigureTopic() (err error) {
|
||||
|
||||
func (p *TopicPublisher) doLookupTopicPartitions() (assignments []*mq_pb.BrokerPartitionAssignment, err error) {
|
||||
if len(p.config.Brokers) == 0 {
|
||||
return nil, fmt.Errorf("no bootstrap brokers")
|
||||
return nil, fmt.Errorf("lookup found no bootstrap brokers")
|
||||
}
|
||||
var lastErr error
|
||||
for _, brokerAddress := range p.config.Brokers {
|
||||
|
||||
@@ -1,7 +1,6 @@
|
||||
package sub_client
|
||||
|
||||
import (
|
||||
"context"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
@@ -12,10 +11,17 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
|
||||
waitTime := 1 * time.Second
|
||||
for {
|
||||
for _, broker := range sub.bootstrapBrokers {
|
||||
|
||||
select {
|
||||
case <-sub.ctx.Done():
|
||||
return
|
||||
default:
|
||||
}
|
||||
|
||||
// lookup topic brokers
|
||||
var brokerLeader string
|
||||
err := pb.WithBrokerGrpcClient(false, broker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
|
||||
resp, err := client.FindBrokerLeader(context.Background(), &mq_pb.FindBrokerLeaderRequest{})
|
||||
resp, err := client.FindBrokerLeader(sub.ctx, &mq_pb.FindBrokerLeaderRequest{})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@@ -30,10 +36,8 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
|
||||
|
||||
// connect to the balancer
|
||||
pb.WithBrokerGrpcClient(true, brokerLeader, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
stream, err := client.SubscriberToSubCoordinator(ctx)
|
||||
stream, err := client.SubscriberToSubCoordinator(sub.ctx)
|
||||
if err != nil {
|
||||
glog.V(0).Infof("subscriber %s: %v", sub.ContentConfig.Topic, err)
|
||||
return err
|
||||
@@ -58,6 +62,13 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
|
||||
|
||||
go func() {
|
||||
for reply := range sub.brokerPartitionAssignmentAckChan {
|
||||
|
||||
select {
|
||||
case <-sub.ctx.Done():
|
||||
return
|
||||
default:
|
||||
}
|
||||
|
||||
glog.V(0).Infof("subscriber instance %s ack %+v", sub.SubscriberConfig.ConsumerGroupInstanceId, reply)
|
||||
if err := stream.Send(reply); err != nil {
|
||||
glog.V(0).Infof("subscriber %s reply: %v", sub.ContentConfig.Topic, err)
|
||||
@@ -73,6 +84,13 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
|
||||
glog.V(0).Infof("subscriber %s receive: %v", sub.ContentConfig.Topic, err)
|
||||
return err
|
||||
}
|
||||
|
||||
select {
|
||||
case <-sub.ctx.Done():
|
||||
return nil
|
||||
default:
|
||||
}
|
||||
|
||||
sub.brokerPartitionAssignmentChan <- resp
|
||||
glog.V(0).Infof("Received assignment: %+v", resp)
|
||||
}
|
||||
|
||||
@@ -2,14 +2,13 @@ package sub_client
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"io"
|
||||
"reflect"
|
||||
"time"
|
||||
)
|
||||
|
||||
type KeyedOffset struct {
|
||||
@@ -35,8 +34,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
if po == nil {
|
||||
po = &schema_pb.PartitionOffset{
|
||||
Partition: assigned.Partition,
|
||||
StartTsNs: time.Now().UnixNano(),
|
||||
StartType: schema_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
|
||||
StartTsNs: sub.ContentConfig.OffsetTsNs,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -47,6 +45,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
|
||||
Topic: sub.ContentConfig.Topic.ToPbTopic(),
|
||||
PartitionOffset: po,
|
||||
OffsetType: sub.ContentConfig.OffsetType,
|
||||
Filter: sub.ContentConfig.Filter,
|
||||
FollowerBroker: assigned.FollowerBroker,
|
||||
SlidingWindowSize: slidingWindowSize,
|
||||
@@ -65,6 +64,9 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case <-sub.ctx.Done():
|
||||
subscribeClient.CloseSend()
|
||||
return
|
||||
case <-stopCh:
|
||||
subscribeClient.CloseSend()
|
||||
return
|
||||
@@ -86,15 +88,27 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
}()
|
||||
|
||||
for {
|
||||
// glog.V(0).Infof("subscriber %s/%s/%s waiting for message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
|
||||
// glog.V(0).Infof("subscriber %s/%s waiting for message", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
|
||||
resp, err := subscribeClient.Recv()
|
||||
if err != nil {
|
||||
if errors.Is(err, io.EOF) {
|
||||
return nil
|
||||
}
|
||||
return fmt.Errorf("subscribe recv: %v", err)
|
||||
}
|
||||
if resp.Message == nil {
|
||||
glog.V(0).Infof("subscriber %s/%s received nil message", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
|
||||
continue
|
||||
}
|
||||
|
||||
select {
|
||||
case <-sub.ctx.Done():
|
||||
return nil
|
||||
case <-stopCh:
|
||||
return nil
|
||||
default:
|
||||
}
|
||||
|
||||
switch m := resp.Message.(type) {
|
||||
case *mq_pb.SubscribeMessageResponse_Data:
|
||||
if m.Data.Ctrl != nil {
|
||||
@@ -102,7 +116,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
continue
|
||||
}
|
||||
if len(m.Data.Key) == 0 {
|
||||
fmt.Printf("empty key %+v, type %v\n", m, reflect.TypeOf(m))
|
||||
// fmt.Printf("empty key %+v, type %v\n", m, reflect.TypeOf(m))
|
||||
continue
|
||||
}
|
||||
onDataMessageFn(m)
|
||||
|
||||
@@ -72,12 +72,12 @@ func (sub *TopicSubscriber) startProcessors() {
|
||||
executors := util.NewLimitedConcurrentExecutor(int(sub.SubscriberConfig.SlidingWindowSize))
|
||||
onDataMessageFn := func(m *mq_pb.SubscribeMessageResponse_Data) {
|
||||
executors.Execute(func() {
|
||||
processErr := sub.OnEachMessageFunc(m.Data.Key, m.Data.Value)
|
||||
if processErr == nil {
|
||||
sub.PartitionOffsetChan <- KeyedOffset{
|
||||
Key: m.Data.Key,
|
||||
Offset: m.Data.TsNs,
|
||||
}
|
||||
if sub.OnDataMessageFunc != nil {
|
||||
sub.OnDataMessageFunc(m)
|
||||
}
|
||||
sub.PartitionOffsetChan <- KeyedOffset{
|
||||
Key: m.Data.Key,
|
||||
Offset: m.Data.TsNs,
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package sub_client
|
||||
|
||||
import (
|
||||
"context"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
@@ -17,50 +18,50 @@ type SubscriberConfiguration struct {
|
||||
SlidingWindowSize int32 // how many messages to process concurrently per partition
|
||||
}
|
||||
|
||||
func (s *SubscriberConfiguration) String() string {
|
||||
return "ClientId: " + s.ClientId + ", ConsumerGroup: " + s.ConsumerGroup + ", ConsumerGroupInstanceId: " + s.ConsumerGroupInstanceId
|
||||
}
|
||||
|
||||
type ContentConfiguration struct {
|
||||
Topic topic.Topic
|
||||
Filter string
|
||||
PartitionOffsets []*schema_pb.PartitionOffset
|
||||
OffsetType schema_pb.OffsetType
|
||||
OffsetTsNs int64
|
||||
}
|
||||
|
||||
type OnDataMessageFn func(m *mq_pb.SubscribeMessageResponse_Data)
|
||||
type OnEachMessageFunc func(key, value []byte) (err error)
|
||||
type OnCompletionFunc func()
|
||||
|
||||
type TopicSubscriber struct {
|
||||
ctx context.Context
|
||||
SubscriberConfig *SubscriberConfiguration
|
||||
ContentConfig *ContentConfiguration
|
||||
brokerPartitionAssignmentChan chan *mq_pb.SubscriberToSubCoordinatorResponse
|
||||
brokerPartitionAssignmentAckChan chan *mq_pb.SubscriberToSubCoordinatorRequest
|
||||
OnDataMessageFnnc OnDataMessageFn
|
||||
OnEachMessageFunc OnEachMessageFunc
|
||||
OnDataMessageFunc OnDataMessageFn
|
||||
OnCompletionFunc OnCompletionFunc
|
||||
bootstrapBrokers []string
|
||||
waitForMoreMessage bool
|
||||
activeProcessors map[topic.Partition]*ProcessorState
|
||||
activeProcessorsLock sync.Mutex
|
||||
PartitionOffsetChan chan KeyedOffset
|
||||
}
|
||||
|
||||
func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration, partitionOffsetChan chan KeyedOffset) *TopicSubscriber {
|
||||
func NewTopicSubscriber(ctx context.Context, bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration, partitionOffsetChan chan KeyedOffset) *TopicSubscriber {
|
||||
return &TopicSubscriber{
|
||||
ctx: ctx,
|
||||
SubscriberConfig: subscriber,
|
||||
ContentConfig: content,
|
||||
brokerPartitionAssignmentChan: make(chan *mq_pb.SubscriberToSubCoordinatorResponse, 1024),
|
||||
brokerPartitionAssignmentAckChan: make(chan *mq_pb.SubscriberToSubCoordinatorRequest, 1024),
|
||||
bootstrapBrokers: bootstrapBrokers,
|
||||
waitForMoreMessage: true,
|
||||
activeProcessors: make(map[topic.Partition]*ProcessorState),
|
||||
PartitionOffsetChan: partitionOffsetChan,
|
||||
}
|
||||
}
|
||||
|
||||
func (sub *TopicSubscriber) SetEachMessageFunc(onEachMessageFn OnEachMessageFunc) {
|
||||
sub.OnEachMessageFunc = onEachMessageFn
|
||||
}
|
||||
|
||||
func (sub *TopicSubscriber) SetOnDataMessageFn(fn OnDataMessageFn) {
|
||||
sub.OnDataMessageFnnc = fn
|
||||
sub.OnDataMessageFunc = fn
|
||||
}
|
||||
|
||||
func (sub *TopicSubscriber) SetCompletionFunc(onCompletionFn OnCompletionFunc) {
|
||||
|
||||
Reference in New Issue
Block a user