merge current message queue code changes (#6201)

* listing files to convert to parquet

* write parquet files

* save logs into parquet files

* pass by value

* compact logs into parquet format

* can skip existing files

* refactor

* refactor

* fix compilation

* when no partition found

* refactor

* add untested parquet file read

* rename package

* refactor

* rename files

* remove unused

* add merged log read func

* parquet wants to know the file size

* rewind by time

* pass in stop ts

* add stop ts

* adjust log

* minor

* adjust log

* skip .parquet files when reading message logs

* skip non message files

* Update subscriber_record.go

* send messages

* skip message data with only ts

* skip non log files

* update parquet-go package

* ensure a valid record type

* add new field to a record type

* Update read_parquet_to_log.go

* fix parquet file name generation

* separating reading parquet and logs

* add key field

* add skipped logs

* use in memory cache

* refactor

* refactor

* refactor

* refactor, and change compact log

* refactor

* rename

* refactor

* fix format

* prefix v to version directory
This commit is contained in:
Chris Lu
2024-11-04 12:08:25 -08:00
committed by GitHub
parent ffe908371d
commit dc784bf217
33 changed files with 1106 additions and 264 deletions

View File

@@ -4,6 +4,7 @@ import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/logstore"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb"
@@ -26,7 +27,7 @@ func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *m
} else {
var localPartition *topic.LocalPartition
if localPartition = b.localTopicManager.GetLocalPartition(t, partition); localPartition == nil {
localPartition = topic.NewLocalPartition(partition, b.genLogFlushFunc(t, assignment.Partition), b.genLogOnDiskReadFunc(t, assignment.Partition))
localPartition = topic.NewLocalPartition(partition, b.genLogFlushFunc(t, partition), logstore.GenMergedReadFunc(b, t, partition))
b.localTopicManager.AddLocalPartition(t, localPartition)
}
}

View File

@@ -67,7 +67,7 @@ func (b *MessageQueueBroker) ConfigureTopic(ctx context.Context, request *mq_pb.
resp.RecordType = request.RecordType
// save the topic configuration on filer
if err := b.fca.SaveTopicConfToFiler(request.Topic, resp); err != nil {
if err := b.fca.SaveTopicConfToFiler(t, resp); err != nil {
return nil, fmt.Errorf("configure topic: %v", err)
}

View File

@@ -2,7 +2,6 @@ package broker
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
@@ -93,9 +92,7 @@ func (b *MessageQueueBroker) PublishFollowMe(stream mq_pb.SeaweedMessaging_Publi
time.Sleep(113 * time.Millisecond)
}
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
partitionGeneration := time.Unix(0, p.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, p.RangeStart, p.RangeStop)
partitionDir := topic.PartitionDir(t, p)
// flush the remaining messages
inMemoryBuffers.CloseInput()

View File

@@ -9,7 +9,6 @@ import (
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
"io"
"time"
)
func (b *MessageQueueBroker) SubscribeFollowMe(stream mq_pb.SeaweedMessaging_SubscribeFollowMeServer) (err error) {
@@ -65,9 +64,7 @@ func (b *MessageQueueBroker) SubscribeFollowMe(stream mq_pb.SeaweedMessaging_Sub
func (b *MessageQueueBroker) readConsumerGroupOffset(initMessage *mq_pb.SubscribeMessageRequest_InitMessage) (offset int64, err error) {
t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.PartitionOffset.Partition)
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
partitionGeneration := time.Unix(0, p.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, p.RangeStart, p.RangeStop)
partitionDir := topic.PartitionDir(t, p)
offsetFileName := fmt.Sprintf("%s.offset", initMessage.ConsumerGroup)
err = b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
@@ -86,9 +83,7 @@ func (b *MessageQueueBroker) readConsumerGroupOffset(initMessage *mq_pb.Subscrib
func (b *MessageQueueBroker) saveConsumerGroupOffset(t topic.Topic, p topic.Partition, consumerGroup string, offset int64) error {
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
partitionGeneration := time.Unix(0, p.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, p.RangeStart, p.RangeStop)
partitionDir := topic.PartitionDir(t, p)
offsetFileName := fmt.Sprintf("%s.offset", consumerGroup)
offsetBytes := make([]byte, 8)

View File

@@ -3,6 +3,7 @@ package broker
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/logstore"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
@@ -40,7 +41,7 @@ func (b *MessageQueueBroker) genLocalPartitionFromFiler(t topic.Topic, partition
self := b.option.BrokerAddress()
for _, assignment := range conf.BrokerPartitionAssignments {
if assignment.LeaderBroker == string(self) && partition.Equals(topic.FromPbPartition(assignment.Partition)) {
localPartition = topic.NewLocalPartition(partition, b.genLogFlushFunc(t, assignment.Partition), b.genLogOnDiskReadFunc(t, assignment.Partition))
localPartition = topic.NewLocalPartition(partition, b.genLogFlushFunc(t, partition), logstore.GenMergedReadFunc(b, t, partition))
b.localTopicManager.AddLocalPartition(t, localPartition)
isGenerated = true
break
@@ -55,7 +56,7 @@ func (b *MessageQueueBroker) ensureTopicActiveAssignments(t topic.Topic, conf *m
hasChanges := pub_balancer.EnsureAssignmentsToActiveBrokers(b.PubBalancer.Brokers, 1, conf.BrokerPartitionAssignments)
if hasChanges {
glog.V(0).Infof("topic %v partition updated assignments: %v", t, conf.BrokerPartitionAssignments)
if err = b.fca.SaveTopicConfToFiler(t.ToPbTopic(), conf); err != nil {
if err = b.fca.SaveTopicConfToFiler(t, conf); err != nil {
return err
}
}

View File

@@ -2,24 +2,15 @@ package broker
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"google.golang.org/protobuf/proto"
"math"
"sync/atomic"
"time"
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
)
func (b *MessageQueueBroker) genLogFlushFunc(t topic.Topic, partition *mq_pb.Partition) log_buffer.LogFlushFuncType {
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
partitionGeneration := time.Unix(0, partition.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, partition.RangeStart, partition.RangeStop)
func (b *MessageQueueBroker) genLogFlushFunc(t topic.Topic, p topic.Partition) log_buffer.LogFlushFuncType {
partitionDir := topic.PartitionDir(t, p)
return func(logBuffer *log_buffer.LogBuffer, startTime, stopTime time.Time, buf []byte) {
if len(buf) == 0 {
@@ -45,7 +36,6 @@ func (b *MessageQueueBroker) genLogFlushFunc(t topic.Topic, partition *mq_pb.Par
b.accessLock.Lock()
defer b.accessLock.Unlock()
p := topic.FromPbPartition(partition)
if localPartition := b.localTopicManager.GetLocalPartition(t, p); localPartition != nil {
localPartition.NotifyLogFlushed(logBuffer.LastFlushTsNs)
}
@@ -53,126 +43,3 @@ func (b *MessageQueueBroker) genLogFlushFunc(t topic.Topic, partition *mq_pb.Par
glog.V(0).Infof("flushing at %d to %s size %d", logBuffer.LastFlushTsNs, targetFile, len(buf))
}
}
func (b *MessageQueueBroker) genLogOnDiskReadFunc(t topic.Topic, partition *mq_pb.Partition) log_buffer.LogReadFromDiskFuncType {
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
partitionGeneration := time.Unix(0, partition.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, partition.RangeStart, partition.RangeStop)
lookupFileIdFn := func(fileId string) (targetUrls []string, err error) {
return b.MasterClient.LookupFileId(fileId)
}
eachChunkFn := func(buf []byte, eachLogEntryFn log_buffer.EachLogEntryFuncType, starTsNs, stopTsNs int64) (processedTsNs int64, err error) {
for pos := 0; pos+4 < len(buf); {
size := util.BytesToUint32(buf[pos : pos+4])
if pos+4+int(size) > len(buf) {
err = fmt.Errorf("LogOnDiskReadFunc: read [%d,%d) from [0,%d)", pos, pos+int(size)+4, len(buf))
return
}
entryData := buf[pos+4 : pos+4+int(size)]
logEntry := &filer_pb.LogEntry{}
if err = proto.Unmarshal(entryData, logEntry); err != nil {
pos += 4 + int(size)
err = fmt.Errorf("unexpected unmarshal mq_pb.Message: %v", err)
return
}
if logEntry.TsNs < starTsNs {
pos += 4 + int(size)
continue
}
if stopTsNs != 0 && logEntry.TsNs > stopTsNs {
println("stopTsNs", stopTsNs, "logEntry.TsNs", logEntry.TsNs)
return
}
if _, err = eachLogEntryFn(logEntry); err != nil {
err = fmt.Errorf("process log entry %v: %v", logEntry, err)
return
}
processedTsNs = logEntry.TsNs
pos += 4 + int(size)
}
return
}
eachFileFn := func(entry *filer_pb.Entry, eachLogEntryFn log_buffer.EachLogEntryFuncType, starTsNs, stopTsNs int64) (processedTsNs int64, err error) {
if len(entry.Content) > 0 {
// skip .offset files
return
}
var urlStrings []string
for _, chunk := range entry.Chunks {
if chunk.Size == 0 {
continue
}
if chunk.IsChunkManifest {
glog.Warningf("this should not happen. unexpected chunk manifest in %s/%s", partitionDir, entry.Name)
return
}
urlStrings, err = lookupFileIdFn(chunk.FileId)
if err != nil {
err = fmt.Errorf("lookup %s: %v", chunk.FileId, err)
return
}
if len(urlStrings) == 0 {
err = fmt.Errorf("no url found for %s", chunk.FileId)
return
}
// try one of the urlString until util.Get(urlString) succeeds
var processed bool
for _, urlString := range urlStrings {
// TODO optimization opportunity: reuse the buffer
var data []byte
if data, _, err = util_http.Get(urlString); err == nil {
processed = true
if processedTsNs, err = eachChunkFn(data, eachLogEntryFn, starTsNs, stopTsNs); err != nil {
return
}
break
}
}
if !processed {
err = fmt.Errorf("no data processed for %s %s", entry.Name, chunk.FileId)
return
}
}
return
}
return func(startPosition log_buffer.MessagePosition, stopTsNs int64, eachLogEntryFn log_buffer.EachLogEntryFuncType) (lastReadPosition log_buffer.MessagePosition, isDone bool, err error) {
startFileName := startPosition.UTC().Format(topic.TIME_FORMAT)
startTsNs := startPosition.Time.UnixNano()
stopTime := time.Unix(0, stopTsNs)
var processedTsNs int64
err = b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
return filer_pb.SeaweedList(client, partitionDir, "", func(entry *filer_pb.Entry, isLast bool) error {
if entry.IsDirectory {
return nil
}
if stopTsNs != 0 && entry.Name > stopTime.UTC().Format(topic.TIME_FORMAT) {
isDone = true
return nil
}
if entry.Name < startPosition.UTC().Format(topic.TIME_FORMAT) {
return nil
}
if processedTsNs, err = eachFileFn(entry, eachLogEntryFn, startTsNs, stopTsNs); err != nil {
return err
}
return nil
}, startFileName, true, math.MaxInt32)
})
lastReadPosition = log_buffer.NewMessagePosition(processedTsNs, -2)
return
}
}