Browse Source
Accumulated changes for message queue (#6600)
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 pathdependabot/go_modules/github.com/parquet-go/parquet-go-0.25.0
committed by
GitHub
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
43 changed files with 1047 additions and 1585 deletions
-
6.gitignore
-
9docker/Makefile
-
BINdocker/agent_pub_record
-
11docker/compose/local-mq-test.yml
-
71other/mq_client_example/agent_pub_record/agent_pub_record.go
-
62other/mq_client_example/agent_sub_record/agent_sub_record.go
-
56other/mq_client_example/example/my_record.go
-
4weed/Makefile
-
1weed/command/command.go
-
5weed/command/mq_agent.go
-
16weed/mq/agent/agent_grpc_pub_session.go
-
15weed/mq/agent/agent_grpc_publish.go
-
57weed/mq/agent/agent_grpc_sub_session.go
-
96weed/mq/agent/agent_grpc_subscribe.go
-
11weed/mq/agent/agent_server.go
-
46weed/mq/broker/broker_grpc_sub.go
-
14weed/mq/client/agent_client/agent_publish.go
-
17weed/mq/client/agent_client/agent_subscribe.go
-
19weed/mq/client/agent_client/publish_session.go
-
56weed/mq/client/agent_client/subscribe_session.go
-
78weed/mq/client/cmd/weed_pub_kv/publisher_kv.go
-
138weed/mq/client/cmd/weed_pub_record/publisher_record.go
-
64weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go
-
99weed/mq/client/cmd/weed_sub_record/subscriber_record.go
-
8weed/mq/client/pub_client/publisher.go
-
10weed/mq/client/pub_client/scheduler.go
-
28weed/mq/client/sub_client/connect_to_sub_coordinator.go
-
26weed/mq/client/sub_client/on_each_partition.go
-
12weed/mq/client/sub_client/subscribe.go
-
23weed/mq/client/sub_client/subscriber.go
-
2weed/mq/logstore/read_log_from_disk.go
-
2weed/mq/logstore/read_parquet_to_log.go
-
2weed/mq/sub_coordinator/market.go
-
8weed/mq/topic/local_partition.go
-
37weed/pb/mq_agent.proto
-
642weed/pb/mq_agent_pb/mq_agent.pb.go
-
84weed/pb/mq_agent_pb/mq_agent_grpc.pb.go
-
7weed/pb/mq_broker.proto
-
444weed/pb/mq_pb/mq_broker.pb.go
-
12weed/pb/mq_schema.proto
-
315weed/pb/schema_pb/mq_schema.pb.go
-
8weed/util/log_buffer/log_buffer.go
-
11weed/util/log_buffer/log_read.go
@ -0,0 +1,62 @@ |
|||||
|
package main |
||||
|
|
||||
|
import ( |
||||
|
"flag" |
||||
|
"fmt" |
||||
|
"github.com/seaweedfs/seaweedfs/other/mq_client_example/example" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/client/agent_client" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/util" |
||||
|
"log" |
||||
|
"time" |
||||
|
) |
||||
|
|
||||
|
var ( |
||||
|
namespace = flag.String("ns", "test", "namespace") |
||||
|
t = flag.String("topic", "test", "topic") |
||||
|
agent = flag.String("agent", "localhost:16777", "mq agent address") |
||||
|
maxPartitionCount = flag.Int("maxPartitionCount", 3, "max partition count") |
||||
|
slidingWindowSize = flag.Int("slidingWindowSize", 1, "per partition concurrency") |
||||
|
timeAgo = flag.Duration("timeAgo", 0, "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") |
||||
|
) |
||||
|
|
||||
|
func main() { |
||||
|
flag.Parse() |
||||
|
|
||||
|
// determine the start of the messages
|
||||
|
var startTsNs int64 |
||||
|
startType := schema_pb.OffsetType_RESUME_OR_EARLIEST |
||||
|
if *timeAgo > 0 { |
||||
|
startTsNs = time.Now().Add(-*timeAgo).UnixNano() |
||||
|
startType = schema_pb.OffsetType_EXACT_TS_NS |
||||
|
} |
||||
|
|
||||
|
session, err := agent_client.NewSubscribeSession(*agent, &agent_client.SubscribeOption{ |
||||
|
ConsumerGroup: "test", |
||||
|
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId), |
||||
|
Topic: topic.NewTopic(*namespace, *t), |
||||
|
OffsetType: startType, |
||||
|
OffsetTsNs: startTsNs, |
||||
|
Filter: "", |
||||
|
MaxSubscribedPartitions: int32(*maxPartitionCount), |
||||
|
SlidingWindowSize: int32(*slidingWindowSize), |
||||
|
}) |
||||
|
if err != nil { |
||||
|
log.Printf("new subscribe session: %v", err) |
||||
|
return |
||||
|
} |
||||
|
defer session.CloseSession() |
||||
|
|
||||
|
counter := 0 |
||||
|
session.SubscribeMessageRecord(func(key []byte, recordValue *schema_pb.RecordValue) { |
||||
|
counter++ |
||||
|
record := example.FromRecordValue(recordValue) |
||||
|
fmt.Printf("%d %s %v\n", counter, string(key), record.Field2) |
||||
|
}, func() { |
||||
|
log.Printf("done received %d messages", counter) |
||||
|
}) |
||||
|
|
||||
|
} |
@ -0,0 +1,56 @@ |
|||||
|
package example |
||||
|
|
||||
|
import ( |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/schema" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" |
||||
|
) |
||||
|
|
||||
|
type MyRecord struct { |
||||
|
Key []byte |
||||
|
Field1 []byte |
||||
|
Field2 string |
||||
|
Field3 int32 |
||||
|
Field4 int64 |
||||
|
Field5 float32 |
||||
|
Field6 float64 |
||||
|
Field7 bool |
||||
|
} |
||||
|
|
||||
|
func MyRecordType() *schema_pb.RecordType { |
||||
|
return 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() |
||||
|
} |
||||
|
|
||||
|
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 FromRecordValue(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(), |
||||
|
} |
||||
|
} |
@ -1,57 +0,0 @@ |
|||||
package agent |
|
||||
|
|
||||
import ( |
|
||||
"context" |
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client" |
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb" |
|
||||
"google.golang.org/grpc" |
|
||||
"google.golang.org/grpc/credentials/insecure" |
|
||||
"math/rand/v2" |
|
||||
"time" |
|
||||
) |
|
||||
|
|
||||
func (a *MessageQueueAgent) StartSubscribeSession(ctx context.Context, req *mq_agent_pb.StartSubscribeSessionRequest) (*mq_agent_pb.StartSubscribeSessionResponse, error) { |
|
||||
sessionId := rand.Int64() |
|
||||
|
|
||||
subscriberConfig := &sub_client.SubscriberConfiguration{ |
|
||||
ConsumerGroup: req.ConsumerGroup, |
|
||||
ConsumerGroupInstanceId: req.ConsumerGroupInstanceId, |
|
||||
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()), |
|
||||
MaxPartitionCount: req.MaxSubscribedPartitions, |
|
||||
SlidingWindowSize: req.SlidingWindowSize, |
|
||||
} |
|
||||
|
|
||||
contentConfig := &sub_client.ContentConfiguration{ |
|
||||
Topic: topic.FromPbTopic(req.Topic), |
|
||||
Filter: req.Filter, |
|
||||
PartitionOffsets: req.PartitionOffsets, |
|
||||
} |
|
||||
|
|
||||
topicSubscriber := sub_client.NewTopicSubscriber( |
|
||||
a.brokersList(), |
|
||||
subscriberConfig, |
|
||||
contentConfig, |
|
||||
make(chan sub_client.KeyedOffset, 1024), |
|
||||
) |
|
||||
|
|
||||
a.subscribersLock.Lock() |
|
||||
// remove inactive publishers to avoid memory leak
|
|
||||
for k, entry := range a.subscribers { |
|
||||
if entry.lastActiveTsNs == 0 { |
|
||||
// this is an active session
|
|
||||
continue |
|
||||
} |
|
||||
if time.Unix(0, entry.lastActiveTsNs).Add(10 * time.Hour).Before(time.Now()) { |
|
||||
delete(a.subscribers, k) |
|
||||
} |
|
||||
} |
|
||||
a.subscribers[SessionId(sessionId)] = &SessionEntry[*sub_client.TopicSubscriber]{ |
|
||||
entry: topicSubscriber, |
|
||||
} |
|
||||
a.subscribersLock.Unlock() |
|
||||
|
|
||||
return &mq_agent_pb.StartSubscribeSessionResponse{ |
|
||||
SessionId: sessionId, |
|
||||
}, nil |
|
||||
} |
|
@ -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) |
|
||||
} |
|
||||
} |
|
@ -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) |
|
||||
} |
|
||||
|
|
||||
} |
|
Write
Preview
Loading…
Cancel
Save
Reference in new issue