diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index b8a41ed97..8c9c9ac4b 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -140,7 +140,7 @@ jobs: max-parallel: 13 matrix: driver: [jetstream] - case: [e2e, diamond-e2e, transformer-e2e, kafka-e2e, http-e2e, nats-e2e, sdks-e2e, reduce-e2e, udsource-e2e, api-e2e, sideinputs-e2e, idle-source-e2e, reduce-sdk-e2e] + case: [e2e, diamond-e2e, transformer-e2e, kafka-e2e, http-e2e, nats-e2e, sdks-e2e, reduce-one-e2e, reduce-two-e2e, udsource-e2e, api-e2e, sideinputs-e2e, idle-source-e2e] steps: - name: Checkout code uses: actions/checkout@v3 diff --git a/Makefile b/Makefile index 34af1282d..f6df2d29c 100644 --- a/Makefile +++ b/Makefile @@ -111,7 +111,8 @@ test-kafka-e2e: test-http-e2e: test-nats-e2e: test-sdks-e2e: -test-reduce-e2e: +test-reduce-one-e2e: +test-reduce-two-e2e: test-api-e2e: test-udsource-e2e: test-transformer-e2e: diff --git a/pkg/apis/numaflow/v1alpha1/const.go b/pkg/apis/numaflow/v1alpha1/const.go index 567fbd670..90768a5dc 100644 --- a/pkg/apis/numaflow/v1alpha1/const.go +++ b/pkg/apis/numaflow/v1alpha1/const.go @@ -153,10 +153,24 @@ const ( // PVC mount path for PBQ PathPBQMount = "/var/numaflow/pbq" - // Default persistent store options - DefaultStoreSyncDuration = 2 * time.Second // Default sync duration for pbq - DefaultStoreMaxBufferSize = 100000 // Default buffer size for pbq in bytes - DefaultStorePath = PathPBQMount + "/wals" // Default store path + // Default WAL options + DefaultWALSyncDuration = 30 * time.Second // Default sync duration for pbq + DefaultWALMaxSyncSize = 5 * 1024 * 1024 // Default size to wait for an explicit sync + DefaultSegmentWALPath = PathPBQMount + "/wals" // Default segment wal path + DefaultWALSegmentRotationDuration = 60 * time.Second // Default segment rotation duration + DefaultWALSegmentSize = 30 * 1024 * 1024 // Default segment size + + // Default GC-events WAL options + DefaultGCEventsWALRotationDuration = 60 * time.Second // Default rotation duration for the GC tracker + DefaultGCEventsWALEventsPath = PathPBQMount + "/events" // Default store path for operations + DefaultGCEventsWALSyncDuration = 30 * time.Second // Default sync duration for the GC tracker + DefaultGCEventsWALRotationEventsCount = 3000 // Default rotation events count for the GC tracker + + // Default WAL Compactor options + DefaultWALCompactorSyncDuration = 30 * time.Second // Default sync duration for the compactor + DefaultWALCompactorMaxFileSize = 30 * 1024 * 1024 // Default max file size for the compactor + DefaultWALCompactionDuration = 60 * time.Second // Default compaction duration + DefaultCompactWALPath = PathPBQMount + "/compact-wals" // Default compaction wal path // DefaultKeyForNonKeyedData Default key for non keyed stream DefaultKeyForNonKeyedData = "NON_KEYED_STREAM" diff --git a/pkg/isb/stores/jetstream/reader_test.go b/pkg/isb/stores/jetstream/reader_test.go index 98c644013..8e33e4fd4 100644 --- a/pkg/isb/stores/jetstream/reader_test.go +++ b/pkg/isb/stores/jetstream/reader_test.go @@ -59,7 +59,7 @@ func TestJetStreamBufferRead(t *testing.T) { defer jw.Close() // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(20), startTime) + messages := testutils.BuildTestWriteMessages(int64(20), startTime, nil) // Verify if buffer is full. for jw.isFull.Load() { select { diff --git a/pkg/isb/stores/jetstream/writer_test.go b/pkg/isb/stores/jetstream/writer_test.go index 7e1f41f5b..5ea9e3d2a 100644 --- a/pkg/isb/stores/jetstream/writer_test.go +++ b/pkg/isb/stores/jetstream/writer_test.go @@ -95,7 +95,7 @@ func TestForwarderJetStreamBuffer(t *testing.T) { defer jw.Close() // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(10), startTime) + messages := testutils.BuildTestWriteMessages(int64(10), startTime, nil) // Verify if buffer is not full. for jw.isFull.Load() { select { @@ -223,7 +223,7 @@ func TestJetStreamBufferWriterBufferFull(t *testing.T) { } // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(2), startTime) + messages := testutils.BuildTestWriteMessages(int64(2), startTime, nil) // Add some data to buffer using write and verify no writes are performed when buffer is full _, errs := jw.Write(ctx, messages) assert.Equal(t, len(errs), 2) @@ -239,7 +239,7 @@ func TestJetStreamBufferWriterBufferFull(t *testing.T) { time.Sleep(500 * time.Millisecond) } } - messages = testutils.BuildTestWriteMessages(int64(2), time.Unix(1636470001, 0)) + messages = testutils.BuildTestWriteMessages(int64(2), time.Unix(1636470001, 0), nil) _, errs = jw.Write(ctx, messages) assert.Equal(t, len(errs), 2) for _, errMsg := range errs { @@ -280,7 +280,7 @@ func TestJetStreamBufferWriterBufferFull_DiscardLatest(t *testing.T) { } // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(2), startTime) + messages := testutils.BuildTestWriteMessages(int64(2), startTime, nil) // Add some data to buffer using write and verify no writes are performed when buffer is full _, errs := jw.Write(ctx, messages) assert.Equal(t, len(errs), 2) @@ -296,7 +296,7 @@ func TestJetStreamBufferWriterBufferFull_DiscardLatest(t *testing.T) { time.Sleep(500 * time.Millisecond) } } - messages = testutils.BuildTestWriteMessages(int64(2), time.Unix(1636470001, 0)) + messages = testutils.BuildTestWriteMessages(int64(2), time.Unix(1636470001, 0), nil) _, errs = jw.Write(ctx, messages) assert.Equal(t, len(errs), 2) for _, errMsg := range errs { diff --git a/pkg/isb/stores/redis/read_test.go b/pkg/isb/stores/redis/read_test.go index 719a0949c..2039e0863 100644 --- a/pkg/isb/stores/redis/read_test.go +++ b/pkg/isb/stores/redis/read_test.go @@ -68,7 +68,7 @@ func TestRedisQRead_Read(t *testing.T) { // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(count, startTime) + messages := testutils.BuildTestWriteMessages(count, startTime, nil) for _, msg := range messages { err := client.Client.XAdd(ctx, &redis.XAddArgs{ Stream: rqr.GetStreamName(), @@ -100,7 +100,7 @@ func TestRedisCheckBacklog(t *testing.T) { // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(count, startTime) + messages := testutils.BuildTestWriteMessages(count, startTime, nil) for _, msg := range messages { err := client.Client.XAdd(ctx, &redis.XAddArgs{ Stream: rqr.GetStreamName(), @@ -392,7 +392,7 @@ func (suite *ReadWritePerformance) TestReadWriteLatency() { suite.False(suite.rqw.IsFull()) var writeMessages = make([]isb.Message, 0, suite.count) - writeMessages = append(writeMessages, testutils.BuildTestWriteMessages(suite.count, testStartTime)...) + writeMessages = append(writeMessages, testutils.BuildTestWriteMessages(suite.count, testStartTime, nil)...) stopped := suite.isdf.Start() @@ -443,7 +443,7 @@ func (suite *ReadWritePerformance) TestReadWriteLatencyPipelining() { suite.False(suite.rqw.IsFull()) var writeMessages = make([]isb.Message, 0, suite.count) - writeMessages = append(writeMessages, testutils.BuildTestWriteMessages(suite.count, testStartTime)...) + writeMessages = append(writeMessages, testutils.BuildTestWriteMessages(suite.count, testStartTime, nil)...) stopped := suite.isdf.Start() diff --git a/pkg/isb/stores/redis/write_test.go b/pkg/isb/stores/redis/write_test.go index b84695c4c..a94f1554d 100644 --- a/pkg/isb/stores/redis/write_test.go +++ b/pkg/isb/stores/redis/write_test.go @@ -223,9 +223,10 @@ func buildTestWriteMessages(rqw *BufferWrite, count int64, startTime time.Time) var messages = make([]isb.Message, 0, count) var internalHashKeysMap map[string]bool var internalHashKeys = make([]string, 0) - messages = append(messages, testutils.BuildTestWriteMessages(count, startTime)...) + messages = append(messages, testutils.BuildTestWriteMessages(count, startTime, nil)...) for i := int64(0); i < count; i++ { tmpTime := startTime.Add(time.Duration(i) * time.Minute) + messages[i].EventTime = tmpTime hashKeyName := rqw.GetHashKeyName(tmpTime) if ok := internalHashKeysMap[hashKeyName]; !ok { internalHashKeys = append(internalHashKeys, hashKeyName) @@ -486,7 +487,7 @@ func TestXTrimOnIsFull(t *testing.T) { // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(10), startTime) + messages := testutils.BuildTestWriteMessages(int64(10), startTime, nil) // Add 10 messages for _, msg := range messages { err := client.Client.XAdd(ctx, &redis.XAddArgs{ @@ -551,7 +552,7 @@ func TestSetWriteInfo(t *testing.T) { // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(10), startTime) + messages := testutils.BuildTestWriteMessages(int64(10), startTime, nil) // Add 10 messages for _, msg := range messages { err := client.Client.XAdd(ctx, &redis.XAddArgs{ diff --git a/pkg/isb/stores/simplebuffer/buffer_test.go b/pkg/isb/stores/simplebuffer/buffer_test.go index d356d6dc4..c52a31538 100644 --- a/pkg/isb/stores/simplebuffer/buffer_test.go +++ b/pkg/isb/stores/simplebuffer/buffer_test.go @@ -38,7 +38,7 @@ func TestNewSimpleBuffer(t *testing.T) { assert.Equal(t, sb.IsEmpty(), true) startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(count, startTime) + writeMessages := testutils.BuildTestWriteMessages(count, startTime, nil) sb.Write(ctx, writeMessages[0:5]) assert.Equal(t, int64(5), sb.writeIdx) assert.Equal(t, int64(0), sb.readIdx) @@ -88,7 +88,7 @@ func TestNewSimpleBuffer_BufferFullWritingStrategyIsDiscard(t *testing.T) { assert.Equal(t, sb.IsEmpty(), true) startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(count, startTime) + writeMessages := testutils.BuildTestWriteMessages(count, startTime, nil) // try to write 3 messages, it should fail (we have only space for 2) // the first 2 messages should be written, the last one should be discarded and returns us a NoRetryableError. diff --git a/pkg/isb/testutils/rw.go b/pkg/isb/testutils/rw.go index 43db24d62..accf096a7 100644 --- a/pkg/isb/testutils/rw.go +++ b/pkg/isb/testutils/rw.go @@ -38,10 +38,13 @@ type PayloadForTest struct { } // BuildTestWriteMessages builds test isb.Message which can be used for testing. -func BuildTestWriteMessages(count int64, startTime time.Time) []isb.Message { +func BuildTestWriteMessages(count int64, startTime time.Time, keys []string) []isb.Message { + if keys == nil { + keys = []string{} + } var messages = make([]isb.Message, 0, count) for i := int64(0); i < count; i++ { - tmpTime := startTime.Add(time.Duration(i) * time.Minute) + tmpTime := startTime.Add(time.Duration(i) * time.Second) result, _ := json.Marshal(PayloadForTest{ Key: fmt.Sprintf("paydload_%d", i), Value: i, @@ -53,7 +56,7 @@ func BuildTestWriteMessages(count int64, startTime time.Time) []isb.Message { EventTime: tmpTime, }, ID: fmt.Sprintf("%d-testVertex-0-0", i), // TODO: hard coded ID suffix ATM, make configurable if needed - Keys: []string{}, + Keys: keys, }, Body: isb.Body{Payload: result}, }, @@ -65,7 +68,7 @@ func BuildTestWriteMessages(count int64, startTime time.Time) []isb.Message { // BuildTestWindowRequests builds test window.TimedWindowRequest which can be used for testing. func BuildTestWindowRequests(count int64, startTime time.Time, windowOp window.Operation) []window.TimedWindowRequest { - var readMessages = BuildTestReadMessages(count, startTime) + var readMessages = BuildTestReadMessages(count, startTime, nil) var windowRequests = make([]window.TimedWindowRequest, count) for idx, readMessage := range readMessages { @@ -78,14 +81,14 @@ func BuildTestWindowRequests(count int64, startTime time.Time, windowOp window.O } // BuildTestReadMessages builds test isb.ReadMessage which can be used for testing. -func BuildTestReadMessages(count int64, startTime time.Time) []isb.ReadMessage { - writeMessages := BuildTestWriteMessages(count, startTime) +func BuildTestReadMessages(count int64, startTime time.Time, keys []string) []isb.ReadMessage { + writeMessages := BuildTestWriteMessages(count, startTime, keys) var readMessages = make([]isb.ReadMessage, count) for idx, writeMessage := range writeMessages { readMessages[idx] = isb.ReadMessage{ Message: writeMessage, - ReadOffset: isb.NewSimpleStringPartitionOffset(fmt.Sprintf("read_%s", writeMessage.Header.ID), 0), + ReadOffset: isb.NewSimpleStringPartitionOffset(fmt.Sprintf("%d", idx), 0), } } @@ -93,8 +96,8 @@ func BuildTestReadMessages(count int64, startTime time.Time) []isb.ReadMessage { } // BuildTestReadMessagesIntOffset builds test isb.ReadMessage which can be used for testing. -func BuildTestReadMessagesIntOffset(count int64, startTime time.Time) []isb.ReadMessage { - writeMessages := BuildTestWriteMessages(count, startTime) +func BuildTestReadMessagesIntOffset(count int64, startTime time.Time, keys []string) []isb.ReadMessage { + writeMessages := BuildTestWriteMessages(count, startTime, keys) var readMessages = make([]isb.ReadMessage, count) for idx, writeMessage := range writeMessages { @@ -103,6 +106,7 @@ func BuildTestReadMessagesIntOffset(count int64, startTime time.Time) []isb.Read readMessages[idx] = isb.ReadMessage{ Message: writeMessage, ReadOffset: isb.NewSimpleIntPartitionOffset(int64(offset), 0), + Watermark: writeMessage.EventTime, } } diff --git a/pkg/isbsvc/redis_service_test.go b/pkg/isbsvc/redis_service_test.go index bcc46e260..444db9eb9 100644 --- a/pkg/isbsvc/redis_service_test.go +++ b/pkg/isbsvc/redis_service_test.go @@ -23,10 +23,11 @@ import ( "testing" "time" - "github.com/numaproj/numaflow/pkg/isb/stores/redis" goredis "github.com/redis/go-redis/v9" "github.com/stretchr/testify/assert" + "github.com/numaproj/numaflow/pkg/isb/stores/redis" + "github.com/numaproj/numaflow/pkg/isb/testutils" redisclient "github.com/numaproj/numaflow/pkg/shared/clients/redis" ) @@ -50,7 +51,7 @@ func TestIsbsRedisSvc_Buffers(t *testing.T) { // Verify // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(10), startTime) + messages := testutils.BuildTestWriteMessages(int64(10), startTime, nil) // Add 10 messages for _, msg := range messages { err := redisClient.Client.XAdd(ctx, &goredis.XAddArgs{ diff --git a/pkg/reduce/data_forward.go b/pkg/reduce/data_forward.go index ff941f40b..dfc62de95 100644 --- a/pkg/reduce/data_forward.go +++ b/pkg/reduce/data_forward.go @@ -44,7 +44,7 @@ import ( "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" "github.com/numaproj/numaflow/pkg/reduce/pnf" "github.com/numaproj/numaflow/pkg/shared/idlehandler" "github.com/numaproj/numaflow/pkg/shared/logging" @@ -70,7 +70,7 @@ type DataForward struct { wmbChecker wmb.WMBChecker // wmbChecker checks if the idle watermark is valid when the len(readMessage) is 0. pbqManager *pbq.Manager whereToDecider forwarder.ToWhichStepDecider - storeManager store.Manager + storeManager wal.Manager of *pnf.Manager opts *Options log *zap.SugaredLogger @@ -82,7 +82,7 @@ func NewDataForward(ctx context.Context, fromBuffer isb.BufferReader, toBuffers map[string][]isb.BufferWriter, pbqManager *pbq.Manager, - storeManager store.Manager, + storeManager wal.Manager, whereToDecider forwarder.ToWhichStepDecider, fw fetch.Fetcher, watermarkPublishers map[string]publish.Publisher, @@ -128,13 +128,7 @@ func (df *DataForward) Start() { for { select { case <-df.ctx.Done(): - - // hard shutdown after timeout - cctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) - defer cancel() - // allow to clean itself up. - df.ShutDown(cctx) - + df.ShutDown(df.ctx) return default: // pass the child context so that the reader can be closed. @@ -145,36 +139,101 @@ func (df *DataForward) Start() { } } -// ReplayPersistedMessages replays persisted messages, because during boot up, it has to replay the data from the persistent store of -// PBQ before it can start reading from ISB. ReplayPersistedMessages will return only after the replay has been completed. +// ReplayPersistedMessages replays persisted messages, because during boot up, it has to replay the WAL from the store, +// before it can start reading from ISB. ReplayPersistedMessages will return only after the replay has been completed. func (df *DataForward) ReplayPersistedMessages(ctx context.Context) error { - // FIXME: fix replay for unaligned windows - if df.windower.Type() == window.Unaligned { - return nil - } + startTime := time.Now() + defer func() { + df.log.Infow("ReplayPersistedMessages completed", "timeTaken", time.Since(startTime).String()) + }() - existingStores, err := df.storeManager.DiscoverStores(ctx) + existingWALs, err := df.storeManager.DiscoverWALs(ctx) if err != nil { return err } - for _, s := range existingStores { + // nothing to replay + if len(existingWALs) == 0 { + return nil + } + + if df.windower.Type() == window.Aligned { + return df.replayForAlignedWindows(ctx, existingWALs) + } else { + return df.replayForUnalignedWindows(ctx, existingWALs) + } +} + +// replayForUnalignedWindows replays the messages from WAL for unaligned windows. +func (df *DataForward) replayForUnalignedWindows(ctx context.Context, discoveredWALs []wal.WAL) error { + // ATM len(discoveredWALs) == 1 since we have only 1 slot + for _, s := range discoveredWALs { + p := s.PartitionID() + // associate the PBQ and PnF + df.associatePBQAndPnF(ctx, p) + } + eg := errgroup.Group{} + df.log.Info("Number of partitions to replay: ", len(discoveredWALs)) + + // replay the messages from each WAL in parallel + // currently we will have only one WAL because we use shared partition + // for unaligned windows if we start using slots then we will have multiple WALs + for _, sr := range discoveredWALs { + df.log.Infow("Replaying messages from partition: ", zap.String("partitionID", sr.PartitionID().String())) + func(ctx context.Context, s wal.WAL) { + eg.Go(func() error { + readCh, errCh := s.Replay() + for { + select { + case <-ctx.Done(): + return nil + case err := <-errCh: + if err != nil { + return err + } + case msg, ok := <-readCh: + if !ok { + return nil + } + windowRequests := df.windower.AssignWindows(msg) + for _, winOp := range windowRequests { + // we don't want to persist the messages again + err := df.writeToPBQ(ctx, winOp, false) + if err != nil { + return err + } + } + } + } + }) + }(ctx, sr) + } + return eg.Wait() +} + +// replayForAlignedWindows replays the messages from WAL for aligned windows +func (df *DataForward) replayForAlignedWindows(ctx context.Context, discoveredWALs []wal.WAL) error { + + // Since for aligned windows, we have a WAL for every partition, so there can be multiple WALs. + // We can replay the messages from each WAL in parallel; to do that we need to first + // create a window for each partition and insert it to the windower, + // so that the window can be closed when the watermark crosses the window. + // then we can replay the messages from each WAL in parallel. + for _, s := range discoveredWALs { p := s.PartitionID() - // create a window for each partition and insert it to the windower - // so that the window can be closed when the watermark - // crosses the window. - var timedWindow = window.NewAlignedTimedWindow(p.Start, p.End, p.Slot) - df.windower.InsertWindow(timedWindow) + df.windower.InsertWindow(window.NewAlignedTimedWindow(p.Start, p.End, p.Slot)) - df.associatePBQAndPnF(ctx, &p) + // associate the PBQ and PnF + df.associatePBQAndPnF(ctx, p) } eg := errgroup.Group{} - df.log.Info("Number of partitions to replay: ", len(existingStores)) - // replay the messages from each store in parallel - for _, sr := range existingStores { - df.log.Info("Replaying messages from partition: ", sr.PartitionID().String()) - func(ctx context.Context, s store.Store) { + df.log.Infow("Number of partitions to replay: ", zap.Int("count", len(discoveredWALs))) + + // replay the messages from each WALs in parallel + for _, sr := range discoveredWALs { + df.log.Infow("Replaying messages from partition: ", zap.String("partitionID", sr.PartitionID().String())) + func(ctx context.Context, s wal.WAL) { eg.Go(func() error { pid := s.PartitionID() readCh, errCh := s.Replay() @@ -190,18 +249,18 @@ func (df *DataForward) ReplayPersistedMessages(ctx context.Context) error { if !ok { return nil } - // TODO: support unaligned windows + tw := window.NewAlignedTimedWindow(pid.Start, pid.End, pid.Slot) request := &window.TimedWindowRequest{ ReadMessage: msg, Operation: window.Append, Windows: []window.TimedWindow{tw}, - ID: &pid, + ID: pid, } // we don't want to persist the messages again // because they are already persisted in the store // so we set persist to false - err = df.writeToPBQ(ctx, request, false) + err := df.writeToPBQ(ctx, request, false) if err != nil { return err } @@ -337,7 +396,7 @@ func (df *DataForward) associatePBQAndPnF(ctx context.Context, partitionID *part // we should create and attach the read side of the loop (PnF) to the partition and then // start process-and-forward (pnf) loop df.of.AsyncSchedulePnF(ctx, partitionID, q) - df.log.Debugw("Successfully Created/Found pbq and started PnF", zap.String("partitionID", partitionID.String())) + df.log.Infow("Successfully Created/Found pbq and started PnF", zap.String("partitionID", partitionID.String())) } return q @@ -474,10 +533,10 @@ func (df *DataForward) shouldDropMessage(message *isb.ReadMessage) bool { nextWinAsSeenByWriter := df.windower.NextWindowToBeClosed() // if there is no window open, drop the message if nextWinAsSeenByWriter == nil || df.windower.Type() == window.Unaligned { - df.log.Warnw("Dropping the late message", zap.Time("eventTime", message.EventTime), zap.Time("watermark", message.Watermark)) + df.log.Debugw("Dropping the late message", zap.Time("eventTime", message.EventTime), zap.Time("watermark", message.Watermark)) return true } else if message.EventTime.Before(nextWinAsSeenByWriter.StartTime()) { // if the message doesn't fall in the next window that is about to be closed drop it. - df.log.Warnw("Dropping the late message", zap.Time("eventTime", message.EventTime), zap.Time("watermark", message.Watermark), zap.Time("nextWindowToBeClosed", nextWinAsSeenByWriter.StartTime())) + df.log.Debugw("Dropping the late message", zap.Time("eventTime", message.EventTime), zap.Time("watermark", message.Watermark), zap.Time("nextWindowToBeClosed", nextWinAsSeenByWriter.StartTime())) metrics.ReduceDroppedMessagesCount.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, @@ -536,7 +595,7 @@ func (df *DataForward) writeToPBQ(ctx context.Context, winOp *window.TimedWindow q := df.associatePBQAndPnF(ctx, winOp.ID) err := wait.ExponentialBackoff(pbqWriteBackoff, func() (done bool, err error) { - rErr := q.Write(context.Background(), winOp, persist) + rErr := q.Write(ctx, winOp, persist) if rErr != nil { df.log.Errorw("Failed to write message", zap.String("msgOffSet", winOp.ReadMessage.ReadOffset.String()), zap.String("partitionID", winOp.ID.String()), zap.Error(rErr)) metrics.PBQWriteErrorCount.With(map[string]string{ diff --git a/pkg/reduce/data_forward_test.go b/pkg/reduce/data_forward_test.go index 1905e5df6..d520feeb3 100644 --- a/pkg/reduce/data_forward_test.go +++ b/pkg/reduce/data_forward_test.go @@ -34,8 +34,8 @@ import ( "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store/aligned/memory" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/aligned/memory" "github.com/numaproj/numaflow/pkg/reduce/pnf" "github.com/numaproj/numaflow/pkg/shared/kvs" "github.com/numaproj/numaflow/pkg/watermark/entity" @@ -388,7 +388,7 @@ func TestDataForward_StartWithNoOpWM(t *testing.T) { } // create store manager - storeManager := memory.NewMemoryStores(memory.WithStoreSize(100)) + storeManager := memory.NewMemManager(memory.WithStoreSize(100)) // create pbqManager pbqManager, err := pbq.NewManager(child, "reduce", pipelineName, 0, storeManager, window.Aligned, pbq.WithReadTimeout(1*time.Second), pbq.WithChannelBufferSize(10)) @@ -478,7 +478,7 @@ func TestReduceDataForward_IdleWM(t *testing.T) { } // create store manager - storeManager := memory.NewMemoryStores(memory.WithStoreSize(1000)) + storeManager := memory.NewMemManager(memory.WithStoreSize(1000)) // create pbq manager var pbqManager *pbq.Manager @@ -694,7 +694,7 @@ func TestReduceDataForward_Count(t *testing.T) { } // create store manager - storeManager := memory.NewMemoryStores(memory.WithStoreSize(1000)) + storeManager := memory.NewMemManager(memory.WithStoreSize(1000)) // create pbq manager var pbqManager *pbq.Manager @@ -779,7 +779,7 @@ func TestReduceDataForward_AllowedLatencyCount(t *testing.T) { } // create store manager - storeManager := memory.NewMemoryStores(memory.WithStoreSize(1000)) + storeManager := memory.NewMemManager(memory.WithStoreSize(1000)) // create pbq manager var pbqManager *pbq.Manager @@ -868,7 +868,7 @@ func TestReduceDataForward_Sum(t *testing.T) { } // create store manager - storeManager := memory.NewMemoryStores(memory.WithStoreSize(1000)) + storeManager := memory.NewMemManager(memory.WithStoreSize(1000)) // create pbq manager var pbqManager *pbq.Manager @@ -954,7 +954,7 @@ func TestReduceDataForward_Max(t *testing.T) { } // create store manager - storeManager := memory.NewMemoryStores(memory.WithStoreSize(1000)) + storeManager := memory.NewMemManager(memory.WithStoreSize(1000)) // create pbq manager var pbqManager *pbq.Manager @@ -1040,7 +1040,7 @@ func TestReduceDataForward_FixedSumWithDifferentKeys(t *testing.T) { } // create store manager - storeManager := memory.NewMemoryStores(memory.WithStoreSize(1000)) + storeManager := memory.NewMemManager(memory.WithStoreSize(1000)) // create pbq manager var pbqManager *pbq.Manager @@ -1147,7 +1147,7 @@ func TestReduceDataForward_SumWithDifferentKeys(t *testing.T) { } // create store manager - storeManager := memory.NewMemoryStores(memory.WithStoreSize(1000)) + storeManager := memory.NewMemManager(memory.WithStoreSize(1000)) // create pbq manager var pbqManager *pbq.Manager @@ -1252,7 +1252,7 @@ func TestReduceDataForward_NonKeyed(t *testing.T) { } // create store manager - storeManager := memory.NewMemoryStores(memory.WithStoreSize(1000)) + storeManager := memory.NewMemManager(memory.WithStoreSize(1000)) // create pbq manager var pbqManager *pbq.Manager @@ -1343,7 +1343,7 @@ func TestDataForward_WithContextClose(t *testing.T) { } // create store manager - storeManager := memory.NewMemoryStores(memory.WithStoreSize(100)) + storeManager := memory.NewMemManager(memory.WithStoreSize(100)) // create pbq manager var pbqManager *pbq.Manager @@ -1394,9 +1394,9 @@ func TestDataForward_WithContextClose(t *testing.T) { } } - var discoveredStores []store.Store + var discoveredStores []wal.WAL for { - discoveredStores, _ = storeManager.DiscoverStores(ctx) + discoveredStores, _ = storeManager.DiscoverWALs(ctx) if len(discoveredStores) == 1 { break @@ -1441,7 +1441,7 @@ func TestReduceDataForward_SumMultiPartitions(t *testing.T) { } // create store manager - storeManager := memory.NewMemoryStores(memory.WithStoreSize(1000)) + storeManager := memory.NewMemManager(memory.WithStoreSize(1000)) // create pbq manager var pbqManager *pbq.Manager diff --git a/pkg/reduce/pbq/pbq.go b/pkg/reduce/pbq/pbq.go index a99d18f21..5dd124a20 100644 --- a/pkg/reduce/pbq/pbq.go +++ b/pkg/reduce/pbq/pbq.go @@ -26,7 +26,7 @@ import ( "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" "github.com/numaproj/numaflow/pkg/window" ) @@ -36,7 +36,7 @@ type PBQ struct { vertexName string pipelineName string vertexReplica int32 - store store.Store + store wal.WAL output chan *window.TimedWindowRequest cob bool // cob to avoid panic in case writes happen after close of book PartitionID partition.ID @@ -71,10 +71,6 @@ func (p *PBQ) Write(ctx context.Context, request *window.TimedWindowRequest, per case p.output <- request: switch request.Operation { case window.Open, window.Append, window.Expand: - // this is a blocking call, ctx.Done() will be ignored. - if p.windowType == window.Unaligned { - return nil - } // during replay we do not have to persist if persist { writeErr = p.store.Write(request.ReadMessage) diff --git a/pkg/reduce/pbq/pbq_test.go b/pkg/reduce/pbq/pbq_test.go index f7d790c94..69ac95159 100644 --- a/pkg/reduce/pbq/pbq_test.go +++ b/pkg/reduce/pbq/pbq_test.go @@ -25,8 +25,8 @@ import ( "github.com/numaproj/numaflow/pkg/isb/testutils" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store/aligned" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store/aligned/memory" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/aligned" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/aligned/memory" "github.com/numaproj/numaflow/pkg/window" ) @@ -40,7 +40,7 @@ func TestPBQ_ReadWrite(t *testing.T) { ctx := context.Background() - qManager, _ := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(storeSize)), + qManager, _ := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemManager(memory.WithStoreSize(storeSize)), window.Aligned, WithChannelBufferSize(int64(buffSize)), WithReadTimeout(1*time.Second)) // write 10 window requests @@ -102,7 +102,7 @@ func Test_PBQReadWithCanceledContext(t *testing.T) { ctx := context.Background() - qManager, err = NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(storeSize)), + qManager, err = NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemManager(memory.WithStoreSize(storeSize)), window.Aligned, WithChannelBufferSize(int64(bufferSize)), WithReadTimeout(1*time.Second)) assert.NoError(t, err) @@ -170,7 +170,7 @@ func TestPBQ_WriteWithStoreFull(t *testing.T) { var err error ctx := context.Background() - qManager, err = NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(storeSize)), + qManager, err = NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemManager(memory.WithStoreSize(storeSize)), window.Aligned, WithChannelBufferSize(int64(buffSize)), WithReadTimeout(1*time.Second)) assert.NoError(t, err) diff --git a/pkg/reduce/pbq/pbqmanager.go b/pkg/reduce/pbq/pbqmanager.go index 511cb5d75..a11b897aa 100644 --- a/pkg/reduce/pbq/pbqmanager.go +++ b/pkg/reduce/pbq/pbqmanager.go @@ -29,7 +29,7 @@ import ( "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" "github.com/numaproj/numaflow/pkg/window" "github.com/numaproj/numaflow/pkg/shared/logging" @@ -40,7 +40,7 @@ type Manager struct { vertexName string pipelineName string vertexReplica int32 - storeProvider store.Manager + storeProvider wal.Manager pbqOptions *options pbqMap map[string]*PBQ log *zap.SugaredLogger @@ -53,7 +53,7 @@ type Manager struct { // NewManager returns new instance of manager // We don't intend this to be called by multiple routines. -func NewManager(ctx context.Context, vertexName string, pipelineName string, vr int32, storeProvider store.Manager, windowType window.Type, opts ...PBQOption) (*Manager, error) { +func NewManager(ctx context.Context, vertexName string, pipelineName string, vr int32, storeProvider wal.Manager, windowType window.Type, opts ...PBQOption) (*Manager, error) { pbqOpts := DefaultOptions() for _, opt := range opts { if opt != nil { @@ -79,7 +79,7 @@ func NewManager(ctx context.Context, vertexName string, pipelineName string, vr // CreateNewPBQ creates new pbq for a partition func (m *Manager) CreateNewPBQ(ctx context.Context, partitionID partition.ID) (ReadWriteCloser, error) { - persistentStore, err := m.storeProvider.CreateStore(ctx, partitionID) + persistentStore, err := m.storeProvider.CreateWAL(ctx, partitionID) if err != nil { return nil, fmt.Errorf("failed to create a PBQ store, %w", err) } @@ -196,7 +196,7 @@ func (m *Manager) deregister(partitionID partition.ID) error { metrics.LabelVertexReplicaIndex: strconv.Itoa(int(m.vertexReplica)), }).Dec() - return m.storeProvider.DeleteStore(partitionID) + return m.storeProvider.DeleteWAL(partitionID) } func (m *Manager) getPBQs() []*PBQ { diff --git a/pkg/reduce/pbq/pbqmanager_test.go b/pkg/reduce/pbq/pbqmanager_test.go index b1fab18db..5a8cd19c5 100644 --- a/pkg/reduce/pbq/pbqmanager_test.go +++ b/pkg/reduce/pbq/pbqmanager_test.go @@ -25,8 +25,8 @@ import ( "github.com/numaproj/numaflow/pkg/isb/testutils" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store/aligned/memory" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store/aligned/noop" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/aligned/memory" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/noop" "github.com/numaproj/numaflow/pkg/window" ) @@ -34,7 +34,7 @@ func TestManager_ListPartitions(t *testing.T) { size := int64(100) ctx := context.Background() - pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(size)), + pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemManager(memory.WithStoreSize(size)), window.Aligned, WithReadTimeout(1*time.Second), WithChannelBufferSize(10)) assert.NoError(t, err) @@ -73,7 +73,7 @@ func TestManager_GetPBQ(t *testing.T) { size := int64(100) var pb1, pb2 ReadWriteCloser ctx := context.Background() - pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(size)), + pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemManager(memory.WithStoreSize(size)), window.Aligned, WithReadTimeout(1*time.Second), WithChannelBufferSize(10)) assert.NoError(t, err) @@ -98,7 +98,7 @@ func TestPBQFlow(t *testing.T) { size := int64(100) ctx := context.Background() - pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(size)), + pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemManager(memory.WithStoreSize(size)), window.Aligned, WithReadTimeout(1*time.Second), WithChannelBufferSize(10)) assert.NoError(t, err) diff --git a/pkg/reduce/pbq/store/unaligned/interfaces.go b/pkg/reduce/pbq/store/unaligned/interfaces.go deleted file mode 100644 index b5db1646b..000000000 --- a/pkg/reduce/pbq/store/unaligned/interfaces.go +++ /dev/null @@ -1 +0,0 @@ -package unaligned diff --git a/pkg/reduce/pbq/store/aligned/errors.go b/pkg/reduce/pbq/wal/aligned/errors.go similarity index 100% rename from pkg/reduce/pbq/store/aligned/errors.go rename to pkg/reduce/pbq/wal/aligned/errors.go diff --git a/pkg/reduce/pbq/store/aligned/fs/bootstrap.go b/pkg/reduce/pbq/wal/aligned/fs/bootstrap.go similarity index 93% rename from pkg/reduce/pbq/store/aligned/fs/bootstrap.go rename to pkg/reduce/pbq/wal/aligned/fs/bootstrap.go index 3780eaf81..418d9f989 100644 --- a/pkg/reduce/pbq/store/aligned/fs/bootstrap.go +++ b/pkg/reduce/pbq/wal/aligned/fs/bootstrap.go @@ -38,11 +38,11 @@ func init() { } // IsCorrupted checks whether the file is corrupt -func (w *WAL) IsCorrupted() bool { +func (w *alignedWAL) IsCorrupted() bool { return w.corrupted } -func (w *WAL) readWALHeader() (*partition.ID, error) { +func (w *alignedWAL) readWALHeader() (*partition.ID, error) { if w.rOffset > 0 { return nil, fmt.Errorf("header has already been read, current readoffset is at %d", w.rOffset) } @@ -62,7 +62,7 @@ func (w *WAL) readWALHeader() (*partition.ID, error) { return id, err } -func (w *WAL) isEnd() bool { +func (w *alignedWAL) isEnd() bool { // TODO: If we are done reading, check that we got the expected // number of entries and return EOF. return w.rOffset >= w.readUpTo @@ -91,9 +91,9 @@ func decodeWALHeader(buf io.Reader) (*partition.ID, error) { }, nil } -// Replay replays the WAL messages, returns a channel to read messages and a channel to read errors. -// channel will be closed after all the messages are read from the WAL. -func (w *WAL) Replay() (<-chan *isb.ReadMessage, <-chan error) { +// Replay replays the alignedWAL messages, returns a channel to read messages and a channel to read errors. +// channel will be closed after all the messages are read from the alignedWAL. +func (w *alignedWAL) Replay() (<-chan *isb.ReadMessage, <-chan error) { messages := make(chan *isb.ReadMessage) errs := make(chan error) diff --git a/pkg/reduce/pbq/store/aligned/fs/doc.go b/pkg/reduce/pbq/wal/aligned/fs/doc.go similarity index 100% rename from pkg/reduce/pbq/store/aligned/fs/doc.go rename to pkg/reduce/pbq/wal/aligned/fs/doc.go diff --git a/pkg/reduce/pbq/store/aligned/fs/manager.go b/pkg/reduce/pbq/wal/aligned/fs/manager.go similarity index 69% rename from pkg/reduce/pbq/store/aligned/fs/manager.go rename to pkg/reduce/pbq/wal/aligned/fs/manager.go index 92818cb5e..87a29464c 100644 --- a/pkg/reduce/pbq/store/aligned/fs/manager.go +++ b/pkg/reduce/pbq/wal/aligned/fs/manager.go @@ -22,15 +22,16 @@ import ( "path/filepath" "strconv" "strings" + "sync" "time" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" ) -type fsWAL struct { +type fsManager struct { storePath string // maxBufferSize max size of batch before it's flushed to store maxBatchSize int64 @@ -39,19 +40,20 @@ type fsWAL struct { pipelineName string vertexName string replicaIndex int32 - activeStores map[string]store.Store + activeWals map[string]wal.WAL + mu sync.RWMutex } -// NewFSManager is a FileSystem Stores Manager. -func NewFSManager(vertexInstance *dfv1.VertexInstance, opts ...Option) store.Manager { - s := &fsWAL{ - storePath: dfv1.DefaultStorePath, - maxBatchSize: dfv1.DefaultStoreMaxBufferSize, - syncDuration: dfv1.DefaultStoreSyncDuration, +// NewFSManager is a FileSystem WAL Manager. +func NewFSManager(vertexInstance *dfv1.VertexInstance, opts ...Option) wal.Manager { + s := &fsManager{ + storePath: dfv1.DefaultSegmentWALPath, + maxBatchSize: dfv1.DefaultWALMaxSyncSize, + syncDuration: dfv1.DefaultWALSyncDuration, pipelineName: vertexInstance.Vertex.Spec.PipelineName, vertexName: vertexInstance.Vertex.Spec.AbstractVertex.Name, replicaIndex: vertexInstance.Replica, - activeStores: make(map[string]store.Store), + activeWals: make(map[string]wal.WAL), } for _, o := range opts { o(s) @@ -59,11 +61,14 @@ func NewFSManager(vertexInstance *dfv1.VertexInstance, opts ...Option) store.Man return s } -// CreateStore creates the FS WAL. -func (ws *fsWAL) CreateStore(_ context.Context, partitionID partition.ID) (store.Store, error) { +// CreateWAL creates the FS alignedWAL. +func (ws *fsManager) CreateWAL(_ context.Context, partitionID partition.ID) (wal.WAL, error) { // check if the store is already present // during crash recovery, we might have already created the store while replaying - if store, ok := ws.activeStores[partitionID.String()]; ok { + ws.mu.RLock() + store, ok := ws.activeWals[partitionID.String()] + ws.mu.RUnlock() + if ok { return store, nil } // Create fs dir if not exist @@ -83,36 +88,43 @@ func (ws *fsWAL) CreateStore(_ context.Context, partitionID partition.ID) (store metrics.LabelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), }).Inc() - return NewWriteOnlyWAL(&partitionID, filePath, ws.maxBatchSize, ws.syncDuration, ws.pipelineName, ws.vertexName, ws.replicaIndex) - + w, err := NewAlignedWriteOnlyWAL(&partitionID, filePath, ws.maxBatchSize, ws.syncDuration, ws.pipelineName, ws.vertexName, ws.replicaIndex) + if err != nil { + return nil, err + } + ws.mu.Lock() + ws.activeWals[w.PartitionID().String()] = w + ws.mu.Unlock() + return w, nil } -// DiscoverStores returns all the stores present in the storePath -func (ws *fsWAL) DiscoverStores(_ context.Context) ([]store.Store, error) { +// DiscoverWALs returns all the WALs present in the storePath +func (ws *fsManager) DiscoverWALs(_ context.Context) ([]wal.WAL, error) { files, err := os.ReadDir(ws.storePath) if os.IsNotExist(err) { - return []store.Store{}, nil + return []wal.WAL{}, nil } else if err != nil { return nil, err } - partitions := make([]store.Store, 0) + partitions := make([]wal.WAL, 0) for _, f := range files { if strings.HasPrefix(f.Name(), SegmentPrefix) && !f.IsDir() { filePath := filepath.Join(ws.storePath, f.Name()) - wal, err := NewReadWriteWAL(filePath, ws.maxBatchSize, ws.syncDuration, ws.pipelineName, ws.vertexName, ws.replicaIndex) + wl, err := NewAlignedReadWriteWAL(filePath, ws.maxBatchSize, ws.syncDuration, ws.pipelineName, ws.vertexName, ws.replicaIndex) if err != nil { return nil, err } - partitions = append(partitions, wal) + partitions = append(partitions, wl) + ws.activeWals[wl.PartitionID().String()] = wl } } return partitions, nil } -// DeleteStore deletes the store for the given partitionID -func (ws *fsWAL) DeleteStore(partitionID partition.ID) error { +// DeleteWAL deletes the wal for the given partitionID +func (ws *fsManager) DeleteWAL(partitionID partition.ID) error { var err error defer func() { if err != nil { @@ -148,5 +160,8 @@ func (ws *fsWAL) DeleteStore(partitionID partition.ID) error { metrics.LabelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), }).Dec() } + ws.mu.Lock() + delete(ws.activeWals, partitionID.String()) + ws.mu.Unlock() return err } diff --git a/pkg/reduce/pbq/store/aligned/fs/manager_test.go b/pkg/reduce/pbq/wal/aligned/fs/manager_test.go similarity index 89% rename from pkg/reduce/pbq/store/aligned/fs/manager_test.go rename to pkg/reduce/pbq/wal/aligned/fs/manager_test.go index 918d84722..d9a743b3a 100644 --- a/pkg/reduce/pbq/store/aligned/fs/manager_test.go +++ b/pkg/reduce/pbq/wal/aligned/fs/manager_test.go @@ -63,21 +63,21 @@ func TestWalStores(t *testing.T) { storeProvider := NewFSManager(vi, WithStorePath(tmp)) for _, partitionID := range partitionIds { - _, err = storeProvider.CreateStore(ctx, partitionID) + _, err = storeProvider.CreateWAL(ctx, partitionID) assert.NoError(t, err) } - discoverStores, err := storeProvider.DiscoverStores(ctx) + discoverStores, err := storeProvider.DiscoverWALs(ctx) assert.NoError(t, err) assert.Len(t, discoverStores, len(partitionIds)) for _, partitionID := range partitionIds { - err = storeProvider.DeleteStore(partitionID) + err = storeProvider.DeleteWAL(partitionID) assert.NoError(t, err) } - discoverStores, err = storeProvider.DiscoverStores(ctx) + discoverStores, err = storeProvider.DiscoverWALs(ctx) assert.NoError(t, err) assert.Len(t, discoverStores, 0) diff --git a/pkg/reduce/pbq/store/aligned/fs/metrics.go b/pkg/reduce/pbq/wal/aligned/fs/metrics.go similarity index 96% rename from pkg/reduce/pbq/store/aligned/fs/metrics.go rename to pkg/reduce/pbq/wal/aligned/fs/metrics.go index 2c5739b02..b5edf75fb 100644 --- a/pkg/reduce/pbq/store/aligned/fs/metrics.go +++ b/pkg/reduce/pbq/wal/aligned/fs/metrics.go @@ -38,7 +38,7 @@ var entriesCount = promauto.NewCounterVec(prometheus.CounterOpts{ var entriesBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "pbq_wal", Name: "wal_entries_bytes_total", - Help: "Total number of bytes written to WAL", + Help: "Total number of bytes written to alignedWAL", }, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) var filesCount = promauto.NewCounterVec(prometheus.CounterOpts{ @@ -69,7 +69,7 @@ var fileSyncWaitTime = promauto.NewSummaryVec(prometheus.SummaryOpts{ var entryWriteLatency = promauto.NewSummaryVec(prometheus.SummaryOpts{ Subsystem: "pbq_wal", Name: "wal_entry_write_latency", - Help: "Entry write time to WAL", + Help: "Entry write time to alignedWAL", }, []string{metrics.LabelPipeline, metrics.LabelVertex, metrics.LabelVertexReplicaIndex}) var entryEncodeLatency = promauto.NewSummaryVec(prometheus.SummaryOpts{ diff --git a/pkg/reduce/pbq/store/aligned/fs/options.go b/pkg/reduce/pbq/wal/aligned/fs/options.go similarity index 71% rename from pkg/reduce/pbq/store/aligned/fs/options.go rename to pkg/reduce/pbq/wal/aligned/fs/options.go index 7d5189c67..bf381e8a3 100644 --- a/pkg/reduce/pbq/store/aligned/fs/options.go +++ b/pkg/reduce/pbq/wal/aligned/fs/options.go @@ -16,27 +16,29 @@ limitations under the License. package fs -import "time" +import ( + "time" +) -type Option func(stores *fsWAL) +type Option func(stores *fsManager) -// WithStorePath sets the WAL store path +// WithStorePath sets the alignedWAL store path func WithStorePath(path string) Option { - return func(stores *fsWAL) { + return func(stores *fsManager) { stores.storePath = path } } -// WithMaxBufferSize sets the WAL buffer max size option +// WithMaxBufferSize sets the alignedWAL buffer max size option func WithMaxBufferSize(size int64) Option { - return func(stores *fsWAL) { + return func(stores *fsManager) { stores.maxBatchSize = size } } -// WithSyncDuration sets the WAL sync duration option +// WithSyncDuration sets the alignedWAL sync duration option func WithSyncDuration(maxDuration time.Duration) Option { - return func(stores *fsWAL) { + return func(stores *fsManager) { stores.syncDuration = maxDuration } } diff --git a/pkg/reduce/pbq/store/aligned/fs/wal_segment.go b/pkg/reduce/pbq/wal/aligned/fs/wal_segment.go similarity index 83% rename from pkg/reduce/pbq/store/aligned/fs/wal_segment.go rename to pkg/reduce/pbq/wal/aligned/fs/wal_segment.go index 812830fe3..12d9d92a9 100644 --- a/pkg/reduce/pbq/store/aligned/fs/wal_segment.go +++ b/pkg/reduce/pbq/wal/aligned/fs/wal_segment.go @@ -29,7 +29,7 @@ import ( "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" ) const ( @@ -43,39 +43,38 @@ var ( errChecksumMismatch = fmt.Errorf("data checksum not match") ) -// WAL implements a write-ahead-log. It represents both reader and writer. This WAL is write heavy and read is -// infrequent, meaning a read will only happen during a boot up. WAL will only have one segment since these are +// alignedWAL implements a write-ahead-log. It represents both reader and writer. This alignedWAL is write heavy and read is +// infrequent, meaning a read will only happen during a boot up. alignedWAL will only have one segment since these are // relatively short-lived. -type WAL struct { +type alignedWAL struct { pipelineName string vertexName string replicaIndex int32 maxBatchSize int64 // maxBatchSize is the maximum size of the batch before we sync the file. syncDuration time.Duration // syncDuration is the duration after which the writer will sync the file. - fp *os.File // fp is the file pointer to the WAL segment + fp *os.File // fp is the file pointer to the alignedWAL segment wOffset int64 // wOffset is the write offset as tracked by the writer rOffset int64 // rOffset is the read offset as tracked when reading. readUpTo int64 // readUpTo is the read offset at which the reader will stop reading. - createTime time.Time // createTime is the timestamp when the WAL segment is created. - closed bool // closed indicates whether the file has been closed + createTime time.Time // createTime is the timestamp when the alignedWAL segment is created. corrupted bool // corrupted indicates whether the data of the file has been corrupted - partitionID *partition.ID // partitionID is the partition ID of the WAL segment + partitionID *partition.ID // partitionID is the partition ID of the alignedWAL segment prevSyncedWOffset int64 // prevSyncedWOffset is the write offset that is already synced as tracked by the writer prevSyncedTime time.Time // prevSyncedTime is the time when the last sync was made numOfUnsyncedMsgs int64 } -// NewWriteOnlyWAL creates a new WAL instance for write-only. This will be used in happy path where we are only -// writing to the WAL. -func NewWriteOnlyWAL(id *partition.ID, +// NewAlignedWriteOnlyWAL creates a new alignedWAL instance for write-only. This will be used in happy path where we are only +// writing to the alignedWAL. +func NewAlignedWriteOnlyWAL(id *partition.ID, filePath string, maxBufferSize int64, syncDuration time.Duration, pipelineName string, vertexName string, - replica int32) (store.Store, error) { + replica int32) (wal.WAL, error) { - wal := &WAL{ + w := &alignedWAL{ pipelineName: pipelineName, vertexName: vertexName, replicaIndex: replica, @@ -97,24 +96,24 @@ func NewWriteOnlyWAL(id *partition.ID, if err != nil { return nil, err } - wal.fp = fp - err = wal.writeWALHeader() + w.fp = fp + err = w.writeWALHeader() if err != nil { return nil, err } - return wal, nil + return w, nil } -// NewReadWriteWAL creates a new WAL instance for read-write. This will be used during boot up where we will be replaying -// the messages from the WAL and then writing to it. -func NewReadWriteWAL(filePath string, +// NewAlignedReadWriteWAL creates a new alignedWAL instance for read-write. This will be used during boot up where we will be replaying +// the messages from the alignedWAL and then writing to it. +func NewAlignedReadWriteWAL(filePath string, maxBufferSize int64, syncDuration time.Duration, pipelineName string, vertexName string, - replica int32) (store.Store, error) { - wal := &WAL{ + replica int32) (wal.WAL, error) { + w := &alignedWAL{ pipelineName: pipelineName, vertexName: vertexName, replicaIndex: replica, @@ -135,31 +134,32 @@ func NewReadWriteWAL(filePath string, if err != nil { return nil, err } - wal.fp = fp + w.fp = fp - // read the partition ID from the WAL header and set it in the WAL. - readPartition, err := wal.readWALHeader() + // read the partition ID from the alignedWAL header and set it in the alignedWAL. + readPartition, err := w.readWALHeader() if err != nil { return nil, err } - wal.partitionID = readPartition + w.partitionID = readPartition // set the read up to the end of the file. stat, err := os.Stat(filePath) if err != nil { return nil, err } - wal.readUpTo = stat.Size() + w.readUpTo = stat.Size() - return wal, nil + return w, nil } -func (w *WAL) PartitionID() partition.ID { - return *w.partitionID +// PartitionID returns the partition ID from the WAL. +func (w *alignedWAL) PartitionID() *partition.ID { + return w.partitionID } -// writeWALHeader writes the WAL header to the file. -func (w *WAL) writeWALHeader() (err error) { +// writeWALHeader writes the alignedWAL header to the file. +func (w *alignedWAL) writeWALHeader() (err error) { defer func() { if err != nil { walErrors.With(map[string]string{ @@ -191,7 +191,7 @@ type walHeaderPreamble struct { SLen int16 } -// readMessageHeaderPreamble is the header for each WAL entry +// readMessageHeaderPreamble is the header for each alignedWAL entry type readMessageHeaderPreamble struct { WaterMark int64 Offset int64 @@ -199,7 +199,7 @@ type readMessageHeaderPreamble struct { Checksum uint32 } -// encodeWALHeader builds the WAL header. WAL header is per WAL and has information to build the WAL partition. +// encodeWALHeader builds the alignedWAL header. alignedWAL header is per alignedWAL and has information to build the alignedWAL partition. // The header is of the following format. // // +--------------------+------------------+------------------+-------------+ @@ -207,7 +207,7 @@ type readMessageHeaderPreamble struct { // +--------------------+------------------+------------------+-------------+ // // We require the slot-len because slot is variadic. -func (w *WAL) encodeWALHeader(id *partition.ID) (buf *bytes.Buffer, err error) { +func (w *alignedWAL) encodeWALHeader(id *partition.ID) (buf *bytes.Buffer, err error) { defer func() { if err != nil { walErrors.With(map[string]string{ @@ -237,8 +237,13 @@ func (w *WAL) encodeWALHeader(id *partition.ID) (buf *bytes.Buffer, err error) { return buf, err } -// encodeWALMessage builds the WAL message. -func (w *WAL) encodeWALMessage(message *isb.ReadMessage) (buf *bytes.Buffer, err error) { +func calculateChecksum(data []byte) uint32 { + crc32q := crc32.MakeTable(IEEE) + return crc32.Checksum(data, crc32q) +} + +// encodeWALMessage builds the alignedWAL message. +func (w *alignedWAL) encodeWALMessage(message *isb.ReadMessage) (buf *bytes.Buffer, err error) { defer func() { if err != nil { walErrors.With(map[string]string{ @@ -280,9 +285,9 @@ func (w *WAL) encodeWALMessage(message *isb.ReadMessage) (buf *bytes.Buffer, err return buf, nil } -// encodeWALMessageHeader creates the header of the WAL message. The header is mainly for checksum, verifying the +// encodeWALMessageHeader creates the header of the alignedWAL message. The header is mainly for checksum, verifying the // correctness of the WALMessage. -func (w *WAL) encodeWALMessageHeader(message *isb.ReadMessage, messageLen int64, checksum uint32) (*bytes.Buffer, error) { +func (w *alignedWAL) encodeWALMessageHeader(message *isb.ReadMessage, messageLen int64, checksum uint32) (*bytes.Buffer, error) { watermark := message.Watermark.UnixMilli() offset, err := message.ReadOffset.Sequence() @@ -312,9 +317,9 @@ func (w *WAL) encodeWALMessageHeader(message *isb.ReadMessage, messageLen int64, return buf, nil } -// encodeWALMessageBody uses ReadMessage.Message field as the body of the WAL message, encodes the +// encodeWALMessageBody uses ReadMessage.Message field as the body of the alignedWAL message, encodes the // ReadMessage.Message, and returns. -func (w *WAL) encodeWALMessageBody(readMsg *isb.ReadMessage) ([]byte, error) { +func (w *alignedWAL) encodeWALMessageBody(readMsg *isb.ReadMessage) ([]byte, error) { msgBinary, err := readMsg.Message.MarshalBinary() if err != nil { walErrors.With(map[string]string{ @@ -328,19 +333,14 @@ func (w *WAL) encodeWALMessageBody(readMsg *isb.ReadMessage) ([]byte, error) { return msgBinary, nil } -func calculateChecksum(data []byte) uint32 { - crc32q := crc32.MakeTable(IEEE) - return crc32.Checksum(data, crc32q) -} - -// Write writes the message to the WAL. The format as follow is +// Write writes the message to the alignedWAL. The format as follow is // // +-------------------+----------------+-----------------+--------------+----------------+ // | watermark (int64) | offset (int64) | msg-len (int64) | CRC (unit32) | message []byte | // +-------------------+----------------+-----------------+--------------+----------------+ // // CRC will be used for detecting ReadMessage corruptions. -func (w *WAL) Write(message *isb.ReadMessage) (err error) { +func (w *alignedWAL) Write(message *isb.ReadMessage) (err error) { defer func() { if err != nil { walErrors.With(map[string]string{ @@ -407,8 +407,8 @@ func (w *WAL) Write(message *isb.ReadMessage) (err error) { return err } -// Close closes the WAL Segment. -func (w *WAL) Close() (err error) { +// Close closes the alignedWAL Segment. +func (w *alignedWAL) Close() (err error) { defer func() { if err != nil { walErrors.With(map[string]string{ @@ -436,8 +436,6 @@ func (w *WAL) Close() (err error) { return err } - w.closed = true - return nil } diff --git a/pkg/reduce/pbq/store/aligned/fs/wal_segment_test.go b/pkg/reduce/pbq/wal/aligned/fs/wal_segment_test.go similarity index 88% rename from pkg/reduce/pbq/store/aligned/fs/wal_segment_test.go rename to pkg/reduce/pbq/wal/aligned/fs/wal_segment_test.go index 6b963bed0..3be42ad4c 100644 --- a/pkg/reduce/pbq/store/aligned/fs/wal_segment_test.go +++ b/pkg/reduce/pbq/wal/aligned/fs/wal_segment_test.go @@ -29,7 +29,7 @@ import ( "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/testutils" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" ) var vi = &dfv1.VertexInstance{ @@ -52,9 +52,9 @@ func Test_writeReadHeader(t *testing.T) { tmp := t.TempDir() stores := NewFSManager(vi, WithStorePath(tmp)) - store, err := stores.CreateStore(context.Background(), id) + store, err := stores.CreateWAL(context.Background(), id) assert.NoError(t, err) - wal := store.(*WAL) + wal := store.(*alignedWAL) fName := wal.fp.Name() assert.NoError(t, err) // read will fail because the file was opened only in write only mode @@ -64,14 +64,14 @@ func Test_writeReadHeader(t *testing.T) { fmt.Println(fName) assert.NoError(t, err) - openWAL, err := NewWriteOnlyWAL(&id, fName, dfv1.DefaultStoreMaxBufferSize, dfv1.DefaultStoreSyncDuration, "testPipeline", "testVertex", 0) + openWAL, err := NewAlignedWriteOnlyWAL(&id, fName, dfv1.DefaultWALMaxSyncSize, dfv1.DefaultWALSyncDuration, "testPipeline", "testVertex", 0) assert.NoError(t, err) // we have already read the header in OpenWAL - _, err = openWAL.(*WAL).readWALHeader() + _, err = openWAL.(*alignedWAL).readWALHeader() assert.Error(t, err) // compare the original ID with read ID - assert.Equal(t, id, *openWAL.(*WAL).partitionID) + assert.Equal(t, id, *openWAL.(*alignedWAL).partitionID) err = openWAL.Close() assert.NoError(t, err) @@ -107,9 +107,9 @@ func Test_encodeDecodeHeader(t *testing.T) { t.Run(tt.name, func(t *testing.T) { tmp := t.TempDir() stores := NewFSManager(vi, WithStorePath(tmp)) - wal, err := stores.CreateStore(context.Background(), *tt.id) + wal, err := stores.CreateWAL(context.Background(), *tt.id) assert.NoError(t, err) - newWal := wal.(*WAL) + newWal := wal.(*alignedWAL) got, err := newWal.encodeWALHeader(tt.id) if !tt.wantErr(t, err, fmt.Sprintf("encodeWALHeader(%v)", tt.id)) { return @@ -132,24 +132,24 @@ func Test_writeReadEntry(t *testing.T) { tmp := t.TempDir() stores := NewFSManager(vi, WithStorePath(tmp)) - wal, err := stores.CreateStore(context.Background(), id) + wal, err := stores.CreateWAL(context.Background(), id) assert.NoError(t, err) startTime := time.Unix(1665109020, 0).In(location) msgCount := 2 - writeMessages := testutils.BuildTestReadMessagesIntOffset(int64(msgCount), startTime) + writeMessages := testutils.BuildTestReadMessagesIntOffset(int64(msgCount), startTime, nil) message := writeMessages[0] err = wal.Write(&message) assert.NoError(t, err) err = wal.Close() assert.NoError(t, err) - // Reopen the WAL for read and write. - discoveredStores, err := stores.DiscoverStores(context.Background()) + // Reopen the alignedWAL for read and write. + discoveredStores, err := stores.DiscoverWALs(context.Background()) assert.Len(t, discoveredStores, 1) assert.NoError(t, err) store := discoveredStores[0] - newWal := store.(*WAL) + newWal := store.(*alignedWAL) // we have already read the header in OpenWAL _, err = newWal.readWALHeader() assert.Error(t, err) @@ -197,7 +197,7 @@ outerLoop: func Test_encodeDecodeEntry(t *testing.T) { // write 1 isb messages to persisted store startTime := time.Unix(1665109020, 0).In(location) - writeMessages := testutils.BuildTestReadMessagesIntOffset(1, startTime) + writeMessages := testutils.BuildTestReadMessagesIntOffset(1, startTime, nil) firstMessage := writeMessages[0] tests := []struct { name string @@ -228,9 +228,9 @@ func Test_encodeDecodeEntry(t *testing.T) { t.Run(tt.name, func(t *testing.T) { tmp := t.TempDir() stores := NewFSManager(vi, WithStorePath(tmp)) - wal, err := stores.CreateStore(context.Background(), partition.ID{}) + wal, err := stores.CreateWAL(context.Background(), partition.ID{}) assert.NoError(t, err) - newWal := wal.(*WAL) + newWal := wal.(*alignedWAL) got, err := newWal.encodeWALMessage(tt.message) if !tt.wantErr(t, err, fmt.Sprintf("encodeWALMessage(%v)", tt.message)) { @@ -262,15 +262,15 @@ func Test_batchSyncWithMaxBatchSize(t *testing.T) { tmp := t.TempDir() stores := NewFSManager(vi, WithStorePath(tmp)) - wal, err := stores.CreateStore(context.Background(), id) + wal, err := stores.CreateWAL(context.Background(), id) assert.NoError(t, err) - tempWAL := wal.(*WAL) + tempWAL := wal.(*alignedWAL) tempWAL.prevSyncedTime = time.Now() startTime := time.Unix(1665109020, 0).In(location) msgCount := 2 - writeMessages := testutils.BuildTestReadMessagesIntOffset(int64(msgCount), startTime) + writeMessages := testutils.BuildTestReadMessagesIntOffset(int64(msgCount), startTime, nil) message := writeMessages[0] err = wal.Write(&message) assert.NoError(t, err) @@ -287,11 +287,11 @@ func Test_batchSyncWithMaxBatchSize(t *testing.T) { err = wal.Close() assert.NoError(t, err) - // Reopen the WAL for read and write. - discoveredStores, err := stores.DiscoverStores(ctx) + // Reopen the alignedWAL for read and write. + discoveredStores, err := stores.DiscoverWALs(ctx) assert.Len(t, discoveredStores, 1) assert.NoError(t, err) - newWal := discoveredStores[0].(*WAL) + newWal := discoveredStores[0].(*alignedWAL) // we have already read the header in OpenWAL _, err = newWal.readWALHeader() assert.Error(t, err) @@ -345,24 +345,24 @@ func Test_batchSyncWithSyncDuration(t *testing.T) { } tmp := t.TempDir() - stores := &fsWAL{ + stores := &fsManager{ storePath: tmp, - maxBatchSize: dfv1.DefaultStoreMaxBufferSize, + maxBatchSize: dfv1.DefaultWALMaxSyncSize, syncDuration: 0, pipelineName: vi.Vertex.Spec.PipelineName, vertexName: vi.Vertex.Spec.AbstractVertex.Name, replicaIndex: vi.Replica, - activeStores: make(map[string]store.Store), + activeWals: make(map[string]wal.WAL), } - wal, err := stores.CreateStore(context.Background(), id) + wal, err := stores.CreateWAL(context.Background(), id) assert.NoError(t, err) - tempWAL := wal.(*WAL) + tempWAL := wal.(*alignedWAL) startTime := time.Unix(1665109020, 0).In(location) msgCount := 2 - writeMessages := testutils.BuildTestReadMessagesIntOffset(int64(msgCount), startTime) + writeMessages := testutils.BuildTestReadMessagesIntOffset(int64(msgCount), startTime, nil) message := writeMessages[0] storePrevSyncedTime := tempWAL.prevSyncedTime err = wal.Write(&message) @@ -380,11 +380,11 @@ func Test_batchSyncWithSyncDuration(t *testing.T) { err = wal.Close() assert.NoError(t, err) - // Reopen the WAL for read and write. - discoverStores, err := stores.DiscoverStores(context.Background()) + // Reopen the alignedWAL for read and write. + discoverStores, err := stores.DiscoverWALs(context.Background()) assert.Len(t, discoverStores, 1) assert.NoError(t, err) - newWal := discoverStores[0].(*WAL) + newWal := discoverStores[0].(*alignedWAL) // we have already read the header in OpenWAL _, err = newWal.readWALHeader() assert.Error(t, err) diff --git a/pkg/reduce/pbq/store/aligned/memory/manager.go b/pkg/reduce/pbq/wal/aligned/memory/manager.go similarity index 78% rename from pkg/reduce/pbq/store/aligned/memory/manager.go rename to pkg/reduce/pbq/wal/aligned/memory/manager.go index 3df6823c5..4272d9e21 100644 --- a/pkg/reduce/pbq/store/aligned/memory/manager.go +++ b/pkg/reduce/pbq/wal/aligned/memory/manager.go @@ -23,19 +23,19 @@ import ( "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" "github.com/numaproj/numaflow/pkg/shared/logging" ) -type memoryStores struct { +type memManager struct { storeSize int64 - discoverFunc func(ctx context.Context) ([]store.Store, error) + discoverFunc func(ctx context.Context) ([]wal.WAL, error) partitions map[partition.ID]*memoryStore sync.RWMutex } -func NewMemoryStores(opts ...Option) store.Manager { - s := &memoryStores{ +func NewMemManager(opts ...Option) wal.Manager { + s := &memManager{ storeSize: 100000, partitions: make(map[partition.ID]*memoryStore), } @@ -46,7 +46,7 @@ func NewMemoryStores(opts ...Option) store.Manager { return s } -func (ms *memoryStores) CreateStore(ctx context.Context, partitionID partition.ID) (store.Store, error) { +func (ms *memManager) CreateWAL(ctx context.Context, partitionID partition.ID) (wal.WAL, error) { ms.Lock() defer ms.Unlock() if memStore, ok := ms.partitions[partitionID]; ok { @@ -65,11 +65,11 @@ func (ms *memoryStores) CreateStore(ctx context.Context, partitionID partition.I return memStore, nil } -func (ms *memoryStores) DiscoverStores(ctx context.Context) ([]store.Store, error) { +func (ms *memManager) DiscoverWALs(ctx context.Context) ([]wal.WAL, error) { ms.RLock() defer ms.RUnlock() if ms.discoverFunc == nil { - s := make([]store.Store, 0) + s := make([]wal.WAL, 0) for _, val := range ms.partitions { s = append(s, val) } @@ -78,7 +78,7 @@ func (ms *memoryStores) DiscoverStores(ctx context.Context) ([]store.Store, erro return ms.discoverFunc(ctx) } -func (ms *memoryStores) DeleteStore(partitionID partition.ID) error { +func (ms *memManager) DeleteWAL(partitionID partition.ID) error { ms.Lock() defer ms.Unlock() memStore, ok := ms.partitions[partitionID] diff --git a/pkg/reduce/pbq/store/aligned/memory/manager_test.go b/pkg/reduce/pbq/wal/aligned/memory/manager_test.go similarity index 79% rename from pkg/reduce/pbq/store/aligned/memory/manager_test.go rename to pkg/reduce/pbq/wal/aligned/memory/manager_test.go index 89463abdc..f6b30173b 100644 --- a/pkg/reduce/pbq/store/aligned/memory/manager_test.go +++ b/pkg/reduce/pbq/wal/aligned/memory/manager_test.go @@ -24,7 +24,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" ) func TestMemoryStores(t *testing.T) { @@ -49,25 +49,25 @@ func TestMemoryStores(t *testing.T) { Slot: "test-3", }, } - storeProvider := NewMemoryStores(WithStoreSize(100)) + storeProvider := NewMemManager(WithStoreSize(100)) for _, partitionID := range partitionIds { - _, err := storeProvider.CreateStore(ctx, partitionID) + _, err := storeProvider.CreateWAL(ctx, partitionID) assert.NoError(t, err) } - var discoveredStores []store.Store - discoveredStores, err = storeProvider.DiscoverStores(ctx) + var discoveredStores []wal.WAL + discoveredStores, err = storeProvider.DiscoverWALs(ctx) assert.NoError(t, err) assert.Len(t, discoveredStores, len(partitionIds)) for _, partitionID := range partitionIds { - err = storeProvider.DeleteStore(partitionID) + err = storeProvider.DeleteWAL(partitionID) assert.NoError(t, err) } - discoveredStores, err = storeProvider.DiscoverStores(ctx) + discoveredStores, err = storeProvider.DiscoverWALs(ctx) assert.NoError(t, err) assert.Len(t, discoveredStores, 0) diff --git a/pkg/reduce/pbq/store/aligned/memory/options.go b/pkg/reduce/pbq/wal/aligned/memory/options.go similarity index 71% rename from pkg/reduce/pbq/store/aligned/memory/options.go rename to pkg/reduce/pbq/wal/aligned/memory/options.go index b3102841e..6b70170a9 100644 --- a/pkg/reduce/pbq/store/aligned/memory/options.go +++ b/pkg/reduce/pbq/wal/aligned/memory/options.go @@ -19,21 +19,21 @@ package memory import ( "context" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" ) -type Option func(stores *memoryStores) +type Option func(stores *memManager) -// WithDiscoverer sets the discover func of memorystores -func WithDiscoverer(f func(ctx context.Context) ([]store.Store, error)) Option { - return func(stores *memoryStores) { +// WithDiscoverer sets the discover func of memory WAL manager +func WithDiscoverer(f func(ctx context.Context) ([]wal.WAL, error)) Option { + return func(stores *memManager) { stores.discoverFunc = f } } // WithStoreSize sets the store size func WithStoreSize(size int64) Option { - return func(stores *memoryStores) { + return func(stores *memManager) { stores.storeSize = size } } diff --git a/pkg/reduce/pbq/store/aligned/memory/wal.go b/pkg/reduce/pbq/wal/aligned/memory/wal.go similarity index 93% rename from pkg/reduce/pbq/store/aligned/memory/wal.go rename to pkg/reduce/pbq/wal/aligned/memory/wal.go index 219b877b3..a488ec0c0 100644 --- a/pkg/reduce/pbq/store/aligned/memory/wal.go +++ b/pkg/reduce/pbq/wal/aligned/memory/wal.go @@ -19,7 +19,7 @@ package memory import ( "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store/aligned" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/aligned" "go.uber.org/zap" ) @@ -72,6 +72,6 @@ func (m *memoryStore) Close() error { return nil } -func (m *memoryStore) PartitionID() partition.ID { - return m.partitionID +func (m *memoryStore) PartitionID() *partition.ID { + return &m.partitionID } diff --git a/pkg/reduce/pbq/store/aligned/memory/wal_test.go b/pkg/reduce/pbq/wal/aligned/memory/wal_test.go similarity index 90% rename from pkg/reduce/pbq/store/aligned/memory/wal_test.go rename to pkg/reduce/pbq/wal/aligned/memory/wal_test.go index 07e424cae..302902e6e 100644 --- a/pkg/reduce/pbq/store/aligned/memory/wal_test.go +++ b/pkg/reduce/pbq/wal/aligned/memory/wal_test.go @@ -37,13 +37,13 @@ func TestMemoryStore_WriteToStore(t *testing.T) { Slot: "new-partition", } // create a store of size 100 (it can store max 100 messages) - memStore, err := NewMemoryStores(WithStoreSize(100)).CreateStore(ctx, partitionID) + memStore, err := NewMemManager(WithStoreSize(100)).CreateWAL(ctx, partitionID) assert.NoError(t, err) //write 10 isb messages to persisted store msgCount := 10 startTime := time.Now() - writeMessages := testutils.BuildTestReadMessages(int64(msgCount), startTime) + writeMessages := testutils.BuildTestReadMessages(int64(msgCount), startTime, nil) for _, msg := range writeMessages { err := memStore.Write(&msg) @@ -61,13 +61,13 @@ func TestMemoryStore_ReadFromStore(t *testing.T) { } // create a store of size 100 (it can store max 100 messages) - memStore, err := NewMemoryStores(WithStoreSize(100)).CreateStore(ctx, partitionID) + memStore, err := NewMemManager(WithStoreSize(100)).CreateWAL(ctx, partitionID) assert.NoError(t, err) //write 10 isb messages to persisted store msgCount := 10 startTime := time.Now() - writeMessages := testutils.BuildTestReadMessages(int64(msgCount), startTime) + writeMessages := testutils.BuildTestReadMessages(int64(msgCount), startTime, nil) for _, msg := range writeMessages { err := memStore.Write(&msg) @@ -108,7 +108,7 @@ func TestEmptyStore_Read(t *testing.T) { Slot: "new-partition", } - memStore, err := NewMemoryStores(WithStoreSize(storeSize)).CreateStore(ctx, partitionID) + memStore, err := NewMemManager(WithStoreSize(storeSize)).CreateWAL(ctx, partitionID) assert.NoError(t, err) msgCh, errCh := memStore.Replay() readMessages := make([]*isb.ReadMessage, 0) @@ -146,13 +146,13 @@ func TestFullStore_Write(t *testing.T) { Slot: "new-partition", } - memStore, err := NewMemoryStores(WithStoreSize(storeSize)).CreateStore(ctx, partitionID) + memStore, err := NewMemManager(WithStoreSize(storeSize)).CreateWAL(ctx, partitionID) assert.NoError(t, err) //write 100 isb messages to persisted store msgCount := 100 startTime := time.Now() - writeMessages := testutils.BuildTestReadMessages(int64(msgCount), startTime) + writeMessages := testutils.BuildTestReadMessages(int64(msgCount), startTime, nil) for _, msg := range writeMessages { err := memStore.Write(&msg) diff --git a/pkg/reduce/pbq/store/interfaces.go b/pkg/reduce/pbq/wal/interfaces.go similarity index 58% rename from pkg/reduce/pbq/store/interfaces.go rename to pkg/reduce/pbq/wal/interfaces.go index cc4596064..4c6f37e9b 100644 --- a/pkg/reduce/pbq/store/interfaces.go +++ b/pkg/reduce/pbq/wal/interfaces.go @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package store +package wal import ( "context" @@ -23,26 +23,26 @@ import ( "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" ) -// Store provides methods to read, write and delete data from the store. -type Store interface { +// WAL provides methods to read, write and delete data from the WAL. +type WAL interface { // Replay to replay persisted messages during startup // returns a channel to read messages and a channel to read errors Replay() (<-chan *isb.ReadMessage, <-chan error) - // Write writes message to persistence store + // Write writes message to the WAL. Write(msg *isb.ReadMessage) error - // PartitionID returns the partition ID of the store - PartitionID() partition.ID - // Close closes store + // PartitionID returns the partition ID of the WAL. + PartitionID() *partition.ID + // Close closes WAL. Close() error } -// Manager defines the interface to manage the stores. +// Manager defines the interface to manage the WALs. type Manager interface { - // CreateStore returns a new store instance. - CreateStore(context.Context, partition.ID) (Store, error) - // DiscoverStores discovers all the existing stores. - // This is used to recover from a crash and replay all the messages from the store. - DiscoverStores(context.Context) ([]Store, error) - // DeleteStore deletes the store - DeleteStore(partition.ID) error + // CreateWAL returns a new WAL instance. + CreateWAL(context.Context, partition.ID) (WAL, error) + // DiscoverWALs discovers all the existing WALs. + // This is used to recover from a restart and replay all the messages from the WAL. + DiscoverWALs(context.Context) ([]WAL, error) + // DeleteWAL deletes the WAL. + DeleteWAL(partition.ID) error } diff --git a/pkg/reduce/pbq/store/aligned/noop/manager.go b/pkg/reduce/pbq/wal/noop/manager.go similarity index 60% rename from pkg/reduce/pbq/store/aligned/noop/manager.go rename to pkg/reduce/pbq/wal/noop/manager.go index cecec6db4..920766314 100644 --- a/pkg/reduce/pbq/store/aligned/noop/manager.go +++ b/pkg/reduce/pbq/wal/noop/manager.go @@ -20,24 +20,24 @@ import ( "context" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" ) -type noopStores struct { +type noopManager struct { } -func NewNoopStores() store.Manager { - return &noopStores{} +func NewNoopStores() wal.Manager { + return &noopManager{} } -func (ns *noopStores) CreateStore(ctx context.Context, partitionID partition.ID) (store.Store, error) { - return &PBQNoOpStore{}, nil +func (ns *noopManager) CreateWAL(ctx context.Context, partitionID partition.ID) (wal.WAL, error) { + return &noopWAL{}, nil } -func (ns *noopStores) DiscoverStores(ctx context.Context) ([]store.Store, error) { - return []store.Store{}, nil +func (ns *noopManager) DiscoverWALs(ctx context.Context) ([]wal.WAL, error) { + return []wal.WAL{}, nil } -func (ns *noopStores) DeleteStore(partitionID partition.ID) error { +func (ns *noopManager) DeleteWAL(partitionID partition.ID) error { return nil } diff --git a/pkg/reduce/pbq/store/aligned/noop/wal.go b/pkg/reduce/pbq/wal/noop/wal.go similarity index 58% rename from pkg/reduce/pbq/store/aligned/noop/wal.go rename to pkg/reduce/pbq/wal/noop/wal.go index 4ffa767d2..80edbed5d 100644 --- a/pkg/reduce/pbq/store/aligned/noop/wal.go +++ b/pkg/reduce/pbq/wal/noop/wal.go @@ -19,31 +19,31 @@ package noop import ( "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" ) -// PBQNoOpStore is a no-op pbq store which does not do any operation but can be safely invoked. -type PBQNoOpStore struct { +// noopWAL is a no-op pbq WAL which does not do any operation but can be safely invoked. +type noopWAL struct { } -var _ store.Store = (*PBQNoOpStore)(nil) +var _ wal.WAL = (*noopWAL)(nil) -func NewPBQNoOpStore() (*PBQNoOpStore, error) { - return &PBQNoOpStore{}, nil +func NewNoOpWAL() (wal.WAL, error) { + return &noopWAL{}, nil } -func (p *PBQNoOpStore) Replay() (<-chan *isb.ReadMessage, <-chan error) { +func (p *noopWAL) Replay() (<-chan *isb.ReadMessage, <-chan error) { return nil, nil } -func (p *PBQNoOpStore) Write(msg *isb.ReadMessage) error { +func (p *noopWAL) Write(msg *isb.ReadMessage) error { return nil } -func (p *PBQNoOpStore) Close() error { +func (p *noopWAL) Close() error { return nil } -func (p *PBQNoOpStore) PartitionID() partition.ID { - return partition.ID{} +func (p *noopWAL) PartitionID() *partition.ID { + return nil } diff --git a/pkg/reduce/pbq/wal/unaligned/fs/common.go b/pkg/reduce/pbq/wal/unaligned/fs/common.go new file mode 100644 index 000000000..c9cdc79eb --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/common.go @@ -0,0 +1,66 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs + +import ( + "os" + "sort" + "strconv" + "strings" +) + +// listFilesInDir lists all filesToReplay after applying the filterStr in the given directory. If a sort function is provided, +// the filesToReplay are sorted using it. +func listFilesInDir(dirPath, filterStr string, sortFunc func([]os.FileInfo)) ([]os.FileInfo, error) { + dir, err := os.Open(dirPath) + if err != nil { + return nil, err + } + defer func(dir *os.File) { + _ = dir.Close() + }(dir) + + files, err := dir.Readdir(-1) + if err != nil { + return nil, err + } + + // filter the filesToReplay based on the filterStr + var cfs []os.FileInfo + for _, file := range files { + if strings.Contains(file.Name(), filterStr) { + continue + } + cfs = append(cfs, file) + } + + // sort the filesToReplay if a sort function is provided + if sortFunc != nil { + sortFunc(cfs) + } + + return cfs, nil +} + +// sortFunc is a function to sort the filesToReplay based on the timestamp in the file name +var sortFunc = func(files []os.FileInfo) { + sort.Slice(files, func(i, j int) bool { + timeI, _ := strconv.ParseInt(strings.Split(files[i].Name(), "-")[1], 10, 64) + timeJ, _ := strconv.ParseInt(strings.Split(files[j].Name(), "-")[1], 10, 64) + return timeI < timeJ + }) +} diff --git a/pkg/reduce/pbq/wal/unaligned/fs/compactor.go b/pkg/reduce/pbq/wal/unaligned/fs/compactor.go new file mode 100644 index 000000000..4d24e3a25 --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/compactor.go @@ -0,0 +1,596 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs + +import ( + "bufio" + "bytes" + "context" + "encoding/binary" + "errors" + "fmt" + "io" + "os" + "path/filepath" + "strconv" + "strings" + "sync" + "time" + + "go.uber.org/zap" + + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/unaligned" + "github.com/numaproj/numaflow/pkg/shared/logging" +) + +const ( + compactedPrefix = "compacted" + compactionInProgress = "current" + "-" + compactedPrefix +) + +// compactor is a compactor for the data filesToReplay +type compactor struct { + partitionID *partition.ID + compactKeyMap map[string]int64 + gcEventsWALPath string + dataSegmentWALPath string + compactedSegWALPath string + currCompactedFile *os.File + compWriteOffset int64 + compBufWriter *bufio.Writer + prevSyncedTime time.Time + maxFileSize int64 + mu sync.Mutex + dc *decoder + ec *encoder + compactionDuration time.Duration + syncDuration time.Duration + stopSignal chan struct{} + doneCh chan struct{} + latestWatermark int64 + log *zap.SugaredLogger +} + +// NewCompactor returns a new WAL compactor instance +func NewCompactor(ctx context.Context, partitionId *partition.ID, gcEventsPath string, dataSegmentWALPath string, compactedSegWALPath string, opts ...CompactorOption) (unaligned.Compactor, error) { + + c := &compactor{ + compactionDuration: dfv1.DefaultWALCompactionDuration, + maxFileSize: dfv1.DefaultWALCompactorMaxFileSize, + syncDuration: dfv1.DefaultWALCompactorSyncDuration, // FIXME(WAL): we need to sync only at the end + partitionID: partitionId, + dataSegmentWALPath: dataSegmentWALPath, + compactedSegWALPath: compactedSegWALPath, + gcEventsWALPath: gcEventsPath, + compactKeyMap: make(map[string]int64), + prevSyncedTime: time.Now(), + mu: sync.Mutex{}, + dc: newDecoder(), + ec: newEncoder(), + compWriteOffset: 0, + compBufWriter: bufio.NewWriter(nil), + doneCh: make(chan struct{}), + stopSignal: make(chan struct{}), + latestWatermark: -1, + log: logging.FromContext(ctx), + } + + for _, opt := range opts { + opt(c) + } + + // Create WAL dir if not exist + var err error + if _, err = os.Stat(c.dataSegmentWALPath); os.IsNotExist(err) { + err = os.Mkdir(c.dataSegmentWALPath, 0755) + if err != nil { + return nil, err + } + } + + // Create event dir if not exist + if _, err = os.Stat(c.gcEventsWALPath); os.IsNotExist(err) { + err = os.Mkdir(c.gcEventsWALPath, 0755) + if err != nil { + return nil, err + } + } + + // Create compacted dir if not exist + if _, err = os.Stat(c.compactedSegWALPath); os.IsNotExist(err) { + err = os.Mkdir(c.compactedSegWALPath, 0755) + if err != nil { + return nil, err + } + } + + // if the file with the compactionInProgress name exists, it means the compactor was stopped + // abruptly we should rename the file, so that it gets considered for replay + currCompactionFileName := filepath.Join(c.compactedSegWALPath, compactionInProgress) + if _, err = os.Stat(currCompactionFileName); err != nil && !os.IsNotExist(err) { + if err = os.Rename(currCompactionFileName, c.getFilePath(c.compactedSegWALPath)); err != nil { + return nil, err + } + } + + // open the first compaction file to write to + if err = c.openCompactionFile(); err != nil { + return nil, err + } + + return c, nil +} + +func (c *compactor) compactOnBootup(ctx context.Context) error { + c.log.Infow("Compacting on bootup") + // get all the GC events filesToReplay + eventFiles, err := listFilesInDir(c.gcEventsWALPath, currentEventsFile, nil) + if err != nil { + return err + } + + // There maybe some events which were not compacted. This could happen if the compactor was stopped (restart), so we + // should compact those before replaying persisted messages. We detect non-compacted filesToReplay by looking at unprocessed + // gc-events in the GC WAL. + if len(eventFiles) != 0 { + c.log.Infow("Compacting unprocessed GC event files", zap.Int("count", len(eventFiles))) + if err = c.compact(ctx, eventFiles); err != nil { + return err + } + // rotate the compaction file because we need to first replay the pending messages + // if we don't rotate the file, the file name will have "current" in it and it + // will not be considered for replay + err = c.rotateCompactionFile() + if err != nil { + return err + } + } + + return nil +} + +// Start starts the compactor. +func (c *compactor) Start(ctx context.Context) error { + // in case of incomplete compaction we should compact the data filesToReplay + // before starting the compactor + if err := c.compactOnBootup(ctx); err != nil { + return err + } + + // NOTE: this go routine will run while replay is running, but shouldn't have any side-effects. + go c.keepCompacting(ctx) + + return nil +} + +// Stop stops the compactor. +func (c *compactor) Stop() error { + var err error + + // send the stop signal + close(c.stopSignal) + c.log.Info("Sent 'close' signal to stop the compactor, waiting for it to stop the 'done' channel") + // wait for the compactor to stop + <-c.doneCh + + // flush the buffer and sync the file + if err = c.flushAndSync(); err != nil { + return err + } + + // Close the current data file + if err = c.currCompactedFile.Close(); err != nil { + return err + } + + // delete the temp WIP file if it's empty + if c.compWriteOffset == 0 { + return os.Remove(c.currCompactedFile.Name()) + } + + // rename the current compaction file to the segment file + if err = os.Rename(c.currCompactedFile.Name(), c.getFilePath(c.compactedSegWALPath)); err != nil { + return err + } + + return err +} + +// getFilePath returns the file path for new file creation +func (c *compactor) getFilePath(storePath string) string { + return filepath.Join(storePath, compactedPrefix+"-"+fmt.Sprintf("%d-%d", time.Now().UnixNano(), c.latestWatermark)) +} + +// keepCompacting keeps compacting the data filesToReplay every compaction duration +func (c *compactor) keepCompacting(ctx context.Context) { + compTimer := time.NewTicker(c.compactionDuration) + for { + select { + case <-ctx.Done(): + close(c.doneCh) + return + case <-c.stopSignal: + close(c.doneCh) + return + case <-compTimer.C: + // get all the events filesToReplay + eventFiles, _ := listFilesInDir(c.gcEventsWALPath, currentEventsFile, nil) + if len(eventFiles) >= 0 { + err := c.compact(ctx, eventFiles) + // TODO: retry, if its not ctx or stop signal error + if err != nil { + c.log.Errorw("Error while compacting", zap.Error(err)) + } + } + } + } + +} + +// compact reads all the events file and constructs the compaction key map +// and then compacts the data filesToReplay based on the compaction key map +func (c *compactor) compact(ctx context.Context, eventFiles []os.FileInfo) error { + startTime := time.Now() + // build the compaction key map + err := c.buildCompactionKeyMap(eventFiles) + if err != nil { + return err + } + + // compact the data filesToReplay based on the compaction key map + err = c.compactDataFiles(ctx) + if err != nil { + return err + } + + // delete the events filesToReplay + for _, eventFile := range eventFiles { + // only delete the event files which has watermark less than the latest watermark of the compactor + // this is to ensure that we don't delete the events which are not compacted + if extractWmFromFileName(eventFile.Name()) < c.latestWatermark { + if err = os.Remove(filepath.Join(c.gcEventsWALPath, eventFile.Name())); err != nil { + return err + } + } + } + + c.log.Infow("Compaction completed", zap.String("duration", time.Since(startTime).String())) + return nil +} + +// buildCompactionKeyMap builds the compaction key map from the GC event filesToReplay. The map's key is the "keys" of the +// window and value is the max end-time for which the data has been forwarded to next vertex. This means we can lookup +// this map to see whether the message can be dropped and not carried to the compacted file because we have closed the +// book and have already forwarded. +func (c *compactor) buildCompactionKeyMap(eventFiles []os.FileInfo) error { + c.compactKeyMap = make(map[string]int64) + + for _, eventFile := range eventFiles { + // read the GC events file + opFile, err := os.Open(filepath.Join(c.gcEventsWALPath, eventFile.Name())) + if err != nil { + return err + } + + // iterate over all the GC events + for { + var cEvent *deletionMessage + cEvent, _, err = c.dc.decodeDeletionMessage(opFile) + if errors.Is(err, io.EOF) { + break + } else if err != nil { + // close the file before returning + _ = opFile.Close() + // return the previous main error, not the close error + return err + } + + // track the max end time for each key + if _, ok := c.compactKeyMap[cEvent.Key]; !ok { + c.compactKeyMap[cEvent.Key] = cEvent.Et + } else { + if c.compactKeyMap[cEvent.Key] < cEvent.Et { + c.compactKeyMap[cEvent.Key] = cEvent.Et + } + } + } + + _ = opFile.Close() + } + + return nil +} + +// compactDataFiles compacts the data filesToReplay +func (c *compactor) compactDataFiles(ctx context.Context) error { + + // if there are no events to compact, return + // during replay this list should be == 0, hence it was okay to start the go routine early on + if len(c.compactKeyMap) == 0 { + return nil + } + + // get all the compacted files + compactedFiles, err := listFilesInDir(c.compactedSegWALPath, currentWALPrefix, sortFunc) + if err != nil { + return err + } + + // get all the segment files + segmentFiles, err := listFilesInDir(c.dataSegmentWALPath, currentWALPrefix, sortFunc) + if err != nil { + return err + } + + // we should consider the compacted files first, since the compacted files will have the oldest data + filesToReplay := make([]string, 0) + for _, compactedFile := range compactedFiles { + filesToReplay = append(filesToReplay, filepath.Join(c.compactedSegWALPath, compactedFile.Name())) + } + for _, dataFile := range segmentFiles { + filesToReplay = append(filesToReplay, filepath.Join(c.dataSegmentWALPath, dataFile.Name())) + } + + if len(filesToReplay) == 0 { + return nil + } + + // iterate over all the data filesToReplay and compact them + for _, filePath := range filesToReplay { + select { + case <-ctx.Done(): + return ctx.Err() + case <-c.stopSignal: + // send error, otherwise the compactor will delete + // the event filesToReplay + return fmt.Errorf("compactor stopped") + default: + c.log.Debugw("Compacting file", zap.String("file", filePath)) + if err = c.compactFile(filePath); err != nil { + return err + } + } + } + + return nil +} + +// updateLatestWm updates the latest watermark value +func (c *compactor) updateLatestWm(fileName string) { + wm := extractWmFromFileName(fileName) + if c.latestWatermark < wm { + c.latestWatermark = wm + } +} + +// extractWmFromFileName extracts the watermark from the file name +func extractWmFromFileName(fileName string) int64 { + // fileName will be of format segment-- or compacted-- or events-- + // we need to extract the watermark from the file name and update the latest watermark value. + var wm int64 + wm, _ = strconv.ParseInt(strings.Split(fileName, "-")[2], 10, 64) + return wm +} + +// compactFile compacts the given data file +// it copies the messages from the data file to the compaction file if the message should not be deleted +// and deletes the data file after compaction +func (c *compactor) compactFile(fp string) error { + // open the data file (read only) + dp, err := os.Open(fp) + if err != nil { + return err + } + + // close the file before returning + defer func(dp *os.File) { + _ = dp.Close() + }(dp) + + // read and decode the unalignedWAL header + _, err = c.dc.decodeHeader(dp) + if err != nil { + if errors.Is(err, io.EOF) { + return nil + } + return err + + } + +readLoop: + for { + // read and decode the unalignedWAL message header + mp, err := c.dc.decodeWALMessageHeader(dp) + if err != nil { + if errors.Is(err, io.EOF) { + break readLoop // end of file reached, break the loop + } + return err + } + + // read the key + key := make([]rune, mp.KeyLen) + err = binary.Read(dp, binary.LittleEndian, &key) + if err != nil { + return err + } + + // read the payload + var payload = make([]byte, mp.MessageLen) + read, err := dp.Read(payload) + if err != nil { + return err + } + + mp.Checksum = calculateChecksum(payload) + + if read != int(mp.MessageLen) { + return fmt.Errorf("expected to read length of %d, but read only %d", mp.MessageLen, read) + } + + // skip deleted messages + // we should copy the message only if the message should not be deleted + if !c.shouldKeepMessage(mp.EventTime, string(key)) { + continue + } + + // write the message to the output file + if err = c.writeToFile(mp, string(key), payload); err != nil { + return err + } + + } + + // delete the file, since it's been compacted + if err = os.Remove(fp); err != nil { + return err + } + // update the latest watermark + c.updateLatestWm(fp) + return nil +} + +// shouldKeepMessage checks if the message should be discarded or not +func (c *compactor) shouldKeepMessage(eventTime int64, key string) bool { + // check if the key is present in the compaction key map + ce, ok := c.compactKeyMap[key] + + // we should not discard the messages which are not older than the max end time + if ok && eventTime >= ce { + return true + } + + return false +} + +// writeToFile writes the message to the compacted file and rotates the file if the max file size is reached +func (c *compactor) writeToFile(header *readMessageHeaderPreamble, key string, payload []byte) error { + buf := new(bytes.Buffer) + // write the header + if err := binary.Write(buf, binary.LittleEndian, header); err != nil { + return err + } + + // write the key + if err := binary.Write(buf, binary.LittleEndian, []rune(key)); err != nil { + return err + } + + // write the payload + if wrote, err := buf.Write(payload); err != nil { + return err + } else if wrote != len(payload) { + return fmt.Errorf("expected to write %d, but wrote only %d", len(payload), wrote) + } + + bytesCount, err := c.compBufWriter.Write(buf.Bytes()) + if err != nil { + return err + } + + // update the write offset + c.compWriteOffset += int64(bytesCount) + + // sync the file if the sync duration is reached + if time.Since(c.prevSyncedTime) > c.syncDuration { + if err = c.flushAndSync(); err != nil { + return err + } + } + + if c.compWriteOffset >= c.maxFileSize { + if err = c.rotateCompactionFile(); err != nil { + return err + } + } + + return nil +} + +// rotateCompactionFile rotates the compaction file +func (c *compactor) rotateCompactionFile() error { + + // flush the buffer and sync the file + if err := c.flushAndSync(); err != nil { + return err + } + + // Close the current data file + if err := c.currCompactedFile.Close(); err != nil { + return err + } + + newFileName := c.getFilePath(c.compactedSegWALPath) + // rename the current compaction file to the segment file + if err := os.Rename(c.currCompactedFile.Name(), newFileName); err != nil { + return err + } + + c.log.Debugw("Rotated compaction file", zap.String("file", newFileName)) + + // Open the next data file + return c.openCompactionFile() +} + +// flushAndSync flushes the buffer and calls fs.sync on the compaction file +func (c *compactor) flushAndSync() error { + if err := c.compBufWriter.Flush(); err != nil { + return err + } + if err := c.currCompactedFile.Sync(); err != nil { + return err + } + + // update the previous synced time + c.prevSyncedTime = time.Now() + return nil +} + +// openCompactionFile opens a new compaction file +func (c *compactor) openCompactionFile() error { + c.log.Debugw("Opening new compaction file") + var err error + + c.currCompactedFile, err = os.OpenFile(filepath.Join(c.compactedSegWALPath, compactionInProgress), os.O_WRONLY|os.O_CREATE, 0644) + if err != nil { + return err + } + + // reset the offset + c.compWriteOffset = 0 + + // reset the write buffer (we started off with a nil interface) + c.compBufWriter.Reset(c.currCompactedFile) + + return c.writeWALHeader() +} + +// writeWALHeader writes the unalignedWAL header to the file. +func (c *compactor) writeWALHeader() error { + header, err := c.ec.encodeHeader(c.partitionID) + if err != nil { + return err + } + wrote, err := c.compBufWriter.Write(header) + if wrote != len(header) { + return fmt.Errorf("expected to write %d, but wrote only %d, %w", len(header), wrote, err) + } + + return err +} diff --git a/pkg/reduce/pbq/wal/unaligned/fs/compactor_test.go b/pkg/reduce/pbq/wal/unaligned/fs/compactor_test.go new file mode 100644 index 000000000..cb62e6241 --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/compactor_test.go @@ -0,0 +1,447 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs + +import ( + "context" + "errors" + "fmt" + "io" + "math" + "os" + "path/filepath" + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/numaproj/numaflow/pkg/isb" + "github.com/numaproj/numaflow/pkg/isb/testutils" + "github.com/numaproj/numaflow/pkg/shared/logging" + "github.com/numaproj/numaflow/pkg/window" +) + +func TestCompactor(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + segmentDir := t.TempDir() + defer func() { + cleanupDir(segmentDir) + }() + + compactDir := t.TempDir() + defer func() { + cleanupDir(compactDir) + }() + + pid := window.SharedUnalignedPartition + // write some data filesToReplay + s, err := NewUnalignedWriteOnlyWAL(ctx, &pid, WithStoreOptions(segmentDir, compactDir)) + assert.NoError(t, err) + + keys := []string{"key-1", "key-2"} + // create read messages + readMessages := testutils.BuildTestReadMessagesIntOffset(300, time.UnixMilli(60000), keys) + + // write the messages + for _, readMessage := range readMessages { + err = s.Write(&readMessage) + assert.NoError(t, err) + } + + eventDir := t.TempDir() + // delete all the files in the directory at the end + defer func() { + cleanupDir(eventDir) + }() + /// write some delete events + tracker, err := NewGCEventsWAL(ctx, WithEventsPath(eventDir), WithGCTrackerSyncDuration(100*time.Millisecond), WithGCTrackerRotationDuration(time.Second)) + assert.NoError(t, err) + + ts := time.UnixMilli(60000) + windows := buildTestWindows(ts, 10, time.Second*10, keys) + for _, timedWindow := range windows { + err = tracker.PersistGCEvent(timedWindow) + assert.NoError(t, err) + } + + err = s.Close() + assert.NoError(t, err) + err = tracker.Close() + assert.NoError(t, err) + + // list all the files in the directory + files, err := os.ReadDir(eventDir) + assert.NoError(t, err) + assert.NotEmpty(t, files) + + // create compactor with the data and event directories + c, err := NewCompactor(ctx, &pid, eventDir, segmentDir, compactDir, WithCompactionDuration(time.Second*5), WithCompactorMaxFileSize(1024*1024*5)) + assert.NoError(t, err) + + err = c.Start(ctx) + assert.NoError(t, err) + + err = c.Stop() + assert.NoError(t, err) + + // list all the files in the directory + files, err = os.ReadDir(segmentDir) + assert.NoError(t, err) + assert.Empty(t, files) + + files, err = os.ReadDir(compactDir) + assert.NoError(t, err) + assert.NotEmpty(t, files) + + // read from file and check if the data is correct + d := newDecoder() + + // read the file + file, err := os.OpenFile(filepath.Join(compactDir, files[0].Name()), os.O_RDONLY, 0644) + assert.NoError(t, err) + + header, err := d.decodeHeader(file) + assert.NoError(t, err) + + assert.Equal(t, int64(0), header.Start.UnixMilli()) + assert.Equal(t, int64(math.MaxInt64), header.End.UnixMilli()) + assert.Equal(t, "slot-0", header.Slot) + + for { + msg, _, err := d.decodeMessage(file) + if err != nil { + if errors.Is(err, io.EOF) { + break + } else { + assert.NoError(t, err) + } + } + if msg.EventTime.Before(windows[len(windows)-1].EndTime()) { + assert.Fail(t, "not compacted") + } + } +} + +func TestReplay_AfterCompaction(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + segmentDir := t.TempDir() + // delete all the files in the directory at the end + defer func() { + cleanupDir(segmentDir) + }() + + compactDir := t.TempDir() + // delete all the files in the directory at the end + defer func() { + cleanupDir(compactDir) + }() + + pid := window.SharedUnalignedPartition + // write some data files + s, err := NewUnalignedWriteOnlyWAL(ctx, &pid, WithStoreOptions(segmentDir, compactDir)) + assert.NoError(t, err) + + keys := []string{"key-1", "key-2"} + // create read messages + readMessages := testutils.BuildTestReadMessagesIntOffset(300, time.UnixMilli(60000), keys) + + // write the messages + for _, readMessage := range readMessages { + err = s.Write(&readMessage) + assert.NoError(t, err) + } + + eventDir := t.TempDir() + // delete all the files in the directory at the end + defer func() { + cleanupDir(eventDir) + }() + /// write some delete events + tracker, err := NewGCEventsWAL(ctx, WithEventsPath(eventDir), WithGCTrackerSyncDuration(100*time.Millisecond), WithGCTrackerRotationDuration(time.Second)) + assert.NoError(t, err) + + ts := time.UnixMilli(60000) + windows := buildTestWindows(ts, 10, time.Second*10, keys) + for _, timedWindow := range windows { + err = tracker.PersistGCEvent(timedWindow) + assert.NoError(t, err) + } + + err = s.Close() + assert.NoError(t, err) + err = tracker.Close() + assert.NoError(t, err) + + // list all the files in the directory + files, err := os.ReadDir(eventDir) + assert.NoError(t, err) + assert.NotEmpty(t, files) + + // create compactor with the data and event directories + c, err := NewCompactor(ctx, &pid, eventDir, segmentDir, compactDir, WithCompactionDuration(time.Second*5), WithCompactorMaxFileSize(1024*1024*5)) + assert.NoError(t, err) + + err = c.Start(ctx) + assert.NoError(t, err) + + err = c.Stop() + assert.NoError(t, err) + + sm := NewFSManager(ctx, segmentDir, compactDir, vertexInstance) + wls, err := sm.DiscoverWALs(ctx) + assert.NoError(t, err) + assert.Len(t, wls, 1) + + wl := wls[0] + + // replay the messages + readCh, errCh := wl.Replay() + replayedMessages := make([]*isb.ReadMessage, 0) +readLoop: + for { + select { + case msg, ok := <-readCh: + if !ok { + break readLoop + } + replayedMessages = append(replayedMessages, msg) + case err := <-errCh: + assert.NoError(t, err) + } + } + assert.NoError(t, err) + // first 100 messages will be compacted + assert.Len(t, replayedMessages, 200) + + // order is important + for i := 0; i < 200; i++ { + assert.Equal(t, readMessages[i+100].EventTime.UnixMilli(), replayedMessages[i].EventTime.UnixMilli()) + } + err = wl.Close() + assert.NoError(t, err) +} + +func TestFilesInDir(t *testing.T) { + dir := t.TempDir() + // delete all the files in the directory at the end + defer func() { + cleanupDir(dir) + }() + + // create some files + for i := 0; i < 10; i++ { + file, err := os.Create(filepath.Join(dir, fmt.Sprintf("file-%d", i))) + assert.NoError(t, err) + err = file.Close() + assert.NoError(t, err) + } + + files, err := listFilesInDir(dir, currentWALPrefix, sortFunc) + assert.NoError(t, err) + assert.Len(t, files, 10) + + // add current file + file, err := os.Create(filepath.Join(dir, "current-segment")) + assert.NoError(t, err) + err = file.Close() + assert.NoError(t, err) + + files, err = listFilesInDir(dir, currentWALPrefix, sortFunc) + assert.NoError(t, err) + assert.Len(t, files, 10) + + // remove all files except current + for i := 0; i < 10; i++ { + err = os.Remove(filepath.Join(dir, fmt.Sprintf("file-%d", i))) + assert.NoError(t, err) + } + + files, err = listFilesInDir(dir, currentWALPrefix, sortFunc) + assert.NoError(t, err) + assert.Len(t, files, 0) + + // add 2 more files at the end + file, err = os.Create(filepath.Join(dir, "file-1")) + assert.NoError(t, err) + err = file.Close() + assert.NoError(t, err) + + file, err = os.Create(filepath.Join(dir, "file-2")) + assert.NoError(t, err) + err = file.Close() + assert.NoError(t, err) + + files, err = listFilesInDir(dir, currentWALPrefix, sortFunc) + assert.NoError(t, err) + assert.Len(t, files, 2) + + // add another current file + file, err = os.Create(filepath.Join(dir, "current-compacted")) + assert.NoError(t, err) + err = file.Close() + assert.NoError(t, err) + + files, err = listFilesInDir(dir, currentWALPrefix, sortFunc) + assert.NoError(t, err) + assert.Len(t, files, 2) + + // remove except current + err = os.Remove(filepath.Join(dir, "file-1")) + assert.NoError(t, err) + + err = os.Remove(filepath.Join(dir, "file-2")) + assert.NoError(t, err) + + files, err = listFilesInDir(dir, currentWALPrefix, sortFunc) + assert.NoError(t, err) + assert.Len(t, files, 0) +} + +func TestCompactor_ContextClose(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + + segmentDir := t.TempDir() + // delete all the files in the directory at the end + defer func() { + cleanupDir(segmentDir) + }() + + compactDir := t.TempDir() + // delete all the files in the directory at the end + defer func() { + cleanupDir(compactDir) + }() + + pid := window.SharedUnalignedPartition + // write some data files + s, err := NewUnalignedWriteOnlyWAL(ctx, &pid, WithStoreOptions(segmentDir, compactDir)) + assert.NoError(t, err) + + keys := []string{"key-1", "key-2"} + // create read messages + readMessages := testutils.BuildTestReadMessagesIntOffset(300, time.UnixMilli(60000), keys) + + // write the messages + for _, readMessage := range readMessages { + err = s.Write(&readMessage) + assert.NoError(t, err) + } + + eventDir := t.TempDir() + // delete all the files in the directory at the end + defer func() { + cleanupDir(eventDir) + }() + /// write some delete events + tracker, err := NewGCEventsWAL(ctx, WithEventsPath(eventDir), WithGCTrackerSyncDuration(100*time.Millisecond), WithGCTrackerRotationDuration(time.Second)) + assert.NoError(t, err) + + ts := time.UnixMilli(60000) + windows := buildTestWindows(ts, 10, time.Second*10, keys) + for _, timedWindow := range windows { + err = tracker.PersistGCEvent(timedWindow) + assert.NoError(t, err) + } + + err = s.Close() + assert.NoError(t, err) + err = tracker.Close() + assert.NoError(t, err) + + // create compactor with the data and event directories + c, err := NewCompactor(ctx, &pid, eventDir, segmentDir, compactDir, WithCompactionDuration(time.Second*5), WithCompactorMaxFileSize(1024*1024*5)) + assert.NoError(t, err) + + err = c.Start(ctx) + assert.NoError(t, err) + + cancel() + files, _ := listFilesInDir(segmentDir, currentWALPrefix, nil) + for _, file := range files { + println(file.Name()) + } + time.Sleep(3 * time.Second) + err = c.Stop() + if err != nil { + println(err.Error()) + } + assert.NoError(t, err) +} + +func Test_buildCompactionKeyMap(t *testing.T) { + ctx := context.Background() + + eventDir := t.TempDir() + // delete all the files in the directory at the end + defer func() { + cleanupDir(eventDir) + }() + /// write some delete events + ewl, err := NewGCEventsWAL(ctx, WithEventsPath(eventDir), WithGCTrackerSyncDuration(100*time.Millisecond), WithGCTrackerRotationDuration(time.Second)) + assert.NoError(t, err) + + testWindows := []window.TimedWindow{ + window.NewUnalignedTimedWindow(time.UnixMilli(60000), time.UnixMilli(60010), "slot-0", []string{"key-1", "key-2"}), + window.NewUnalignedTimedWindow(time.UnixMilli(60010), time.UnixMilli(60020), "slot-0", []string{"key-3", "key-4"}), + window.NewUnalignedTimedWindow(time.UnixMilli(60020), time.UnixMilli(60030), "slot-0", []string{"key-5", "key-6"}), + window.NewUnalignedTimedWindow(time.UnixMilli(60030), time.UnixMilli(60040), "slot-0", []string{"", ""}), + window.NewUnalignedTimedWindow(time.UnixMilli(60040), time.UnixMilli(60050), "slot-0", []string{"key-7", "key-8"}), + window.NewUnalignedTimedWindow(time.UnixMilli(60050), time.UnixMilli(60060), "slot-0", []string{"key-5", "key-7"}), + window.NewUnalignedTimedWindow(time.UnixMilli(60060), time.UnixMilli(60070), "slot-0", []string{"key-1", "key-2"}), + window.NewUnalignedTimedWindow(time.UnixMilli(60070), time.UnixMilli(60080), "slot-0", []string{""}), + window.NewUnalignedTimedWindow(time.UnixMilli(60090), time.UnixMilli(60100), "slot-0", []string{"", "", ""}), + } + + for _, timedWindow := range testWindows { + err = ewl.PersistGCEvent(timedWindow) + assert.NoError(t, err) + } + + err = ewl.Close() + assert.NoError(t, err) + + c := &compactor{ + compactKeyMap: make(map[string]int64), + gcEventsWALPath: eventDir, + dc: newDecoder(), + log: logging.FromContext(ctx), + } + + eFiles, err := listFilesInDir(eventDir, currentWALPrefix, nil) + assert.NoError(t, err) + assert.NotEmpty(t, eFiles) + + err = c.buildCompactionKeyMap(eFiles) + assert.NoError(t, err) + + assert.Len(t, c.compactKeyMap, 8) + assert.Equal(t, int64(60030), c.compactKeyMap["key-5:key-6"]) + assert.Equal(t, int64(60050), c.compactKeyMap["key-7:key-8"]) + assert.Equal(t, int64(60060), c.compactKeyMap["key-5:key-7"]) + assert.Equal(t, int64(60070), c.compactKeyMap["key-1:key-2"]) + assert.Equal(t, int64(60040), c.compactKeyMap[":"]) + assert.Equal(t, int64(60100), c.compactKeyMap["::"]) +} + +// cleanup_dir removes all the files in the directory +func cleanupDir(dir string) { + _ = os.RemoveAll(dir) +} diff --git a/pkg/reduce/pbq/wal/unaligned/fs/decoder.go b/pkg/reduce/pbq/wal/unaligned/fs/decoder.go new file mode 100644 index 000000000..f9e803425 --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/decoder.go @@ -0,0 +1,179 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs + +import ( + "encoding/binary" + "fmt" + "io" + "time" + + "github.com/numaproj/numaflow/pkg/isb" + "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" +) + +const ( + dMessageHeaderSize = 22 +) + +var ( + location *time.Location + errChecksumMismatch = fmt.Errorf("data checksum not match") +) + +func init() { + var err error + location, err = time.LoadLocation("UTC") + if err != nil { + panic(fmt.Sprint("cannot load UTC", err)) + } +} + +// decoder is a decoder for the unalignedWAL entries and header. +type decoder struct{} + +// newDecoder returns a new decoder +func newDecoder() *decoder { + return &decoder{} +} + +// decodeHeader decodes the header from the given io.Reader. +// the header of the unalignedWAL file is a partition.ID, so it returns a partition.ID. +func (d *decoder) decodeHeader(buf io.Reader) (*partition.ID, error) { + var err error + + // read the fixed values + var hp = new(walHeaderPreamble) + err = binary.Read(buf, binary.LittleEndian, hp) + if err != nil { + return nil, err + } + + // read the variadic slot + var slot = make([]rune, hp.SLen) + err = binary.Read(buf, binary.LittleEndian, slot) + if err != nil { + return nil, err + } + + return &partition.ID{ + Start: time.UnixMilli(hp.S).In(location), + End: time.UnixMilli(hp.E).In(location), + Slot: string(slot), + }, nil +} + +// decodeMessage decodes the isb read message from the given io.Reader. +func (d *decoder) decodeMessage(buf io.Reader) (*isb.ReadMessage, string, error) { + entryHeader, err := d.decodeWALMessageHeader(buf) + if err != nil { + return nil, "", err + } + + // read the key + key := make([]rune, entryHeader.KeyLen) + err = binary.Read(buf, binary.LittleEndian, &key) + if err != nil { + return nil, "", err + } + + entryBody, err := d.decodeWALBody(buf, entryHeader) + if err != nil { + return nil, "", err + } + + return &isb.ReadMessage{ + Message: *entryBody, + Watermark: time.UnixMilli(entryHeader.WaterMark).In(location), + ReadOffset: isb.SimpleIntOffset(func() int64 { return entryHeader.Offset }), + }, string(key), nil +} + +// decodeDeletionMessage decodes deletion message from the given io.Reader +func (d *decoder) decodeDeletionMessage(buf io.Reader) (*deletionMessage, int64, error) { + dms := deletionMessage{} + + dMessageHeader := new(deletionMessageHeaderPreamble) + if err := binary.Read(buf, binary.LittleEndian, dMessageHeader); err != nil { + return nil, 0, err + } + + dms.St = dMessageHeader.St + dms.Et = dMessageHeader.Et + + // read the slot + var slot = make([]rune, dMessageHeader.SLen) + if err := binary.Read(buf, binary.LittleEndian, slot); err != nil { + return nil, 0, err + } + + dms.Slot = string(slot) + + // read the key + var key = make([]rune, dMessageHeader.KLen) + if err := binary.Read(buf, binary.LittleEndian, key); err != nil { + return nil, 0, err + } + + dms.Key = string(key) + + // compare the checksum + checksum := calculateChecksum([]byte(fmt.Sprintf("%d:%d:%s:%s", dms.St, dms.Et, dms.Slot, dms.Key))) + + if checksum != dMessageHeader.Checksum { + return nil, 0, errChecksumMismatch + } + + size := dMessageHeaderSize + int64(dMessageHeader.SLen) + int64(dMessageHeader.KLen) + return &dms, size, nil +} + +// decodeWALMessageHeader decodes the unalignedWAL message header from the given io.Reader. +func (d *decoder) decodeWALMessageHeader(buf io.Reader) (*readMessageHeaderPreamble, error) { + var entryHeader = new(readMessageHeaderPreamble) + err := binary.Read(buf, binary.LittleEndian, entryHeader) + if err != nil { + return nil, err + } + return entryHeader, nil +} + +// decodeWALBody decodes the unalignedWAL message body from the given io.Reader. +func (d *decoder) decodeWALBody(buf io.Reader, entryHeader *readMessageHeaderPreamble) (*isb.Message, error) { + var err error + + body := make([]byte, entryHeader.MessageLen) + size, err := buf.Read(body) + if err != nil { + return nil, err + } + if int64(size) != entryHeader.MessageLen { + return nil, fmt.Errorf("expected to read length of %d, but wrote only %d", entryHeader.MessageLen, size) + } + // verify the checksum + checksum := calculateChecksum(body) + if checksum != entryHeader.Checksum { + return nil, errChecksumMismatch + } + + var message = new(isb.Message) + err = message.UnmarshalBinary(body) + if err != nil { + return nil, err + } + return message, nil +} diff --git a/pkg/reduce/pbq/wal/unaligned/fs/doc.go b/pkg/reduce/pbq/wal/unaligned/fs/doc.go new file mode 100644 index 000000000..207229cff --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/doc.go @@ -0,0 +1,18 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs implements write-ahead-log on FileSystem. +package fs diff --git a/pkg/reduce/pbq/wal/unaligned/fs/encoder.go b/pkg/reduce/pbq/wal/unaligned/fs/encoder.go new file mode 100644 index 000000000..d675f2a61 --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/encoder.go @@ -0,0 +1,220 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs + +import ( + "bytes" + "encoding/binary" + "fmt" + "hash/crc32" + "log" + "strings" + + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/isb" + "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" +) + +const _IEEE = 0xedb88320 + +// walHeaderPreamble is the header preamble (excludes variadic key) +type walHeaderPreamble struct { + S int64 + E int64 + SLen int16 +} + +// readMessageHeaderPreamble is the header for each unalignedWAL entry +type readMessageHeaderPreamble struct { + EventTime int64 + WaterMark int64 + Offset int64 + MessageLen int64 + KeyLen int32 + Checksum uint32 +} + +// deletionMessageHeaderPreamble is the header for each deletion event. +type deletionMessageHeaderPreamble struct { + St int64 + Et int64 + SLen int16 + KLen int32 + Checksum uint32 +} + +// deletionMessage is the deletion event for a keyed window built from GC events. +type deletionMessage struct { + St int64 + Et int64 + Slot string + Key string +} + +// encoder is an encoder for the unalignedWAL entries and header. +type encoder struct{} + +// newEncoder returns a new encoder +func newEncoder() *encoder { + return &encoder{} +} + +// encodeHeader encodes the header of the unalignedWAL file. +func (e *encoder) encodeHeader(id *partition.ID) ([]byte, error) { + buf := new(bytes.Buffer) + hp := walHeaderPreamble{ + S: id.Start.UnixMilli(), + E: id.End.UnixMilli(), + SLen: int16(len(id.Slot)), + } + + // write the fixed values + if err := binary.Write(buf, binary.LittleEndian, hp); err != nil { + return nil, err + } + + // write the slot + if err := binary.Write(buf, binary.LittleEndian, []rune(id.Slot)); err != nil { + return nil, err + } + return buf.Bytes(), nil +} + +// encodeMessage encodes the given isb.ReadMessage to a binary format. +func (e *encoder) encodeMessage(message *isb.ReadMessage) ([]byte, error) { + buf := new(bytes.Buffer) + + combinedKey := strings.Join(message.Keys, dfv1.KeysDelimitter) + + // Convert the message body to a binary format + body, err := e.encodeWALMessageBody(message) + if err != nil { + return nil, err // return if there is any error + } + + // Calculate checksum of the body + checksum := calculateChecksum(body) + + // Prepare and encode the message header + headerBuf, err := e.encodeWALMessageHeader(message, int64(len(body)), checksum, int32(len(combinedKey))) + if err != nil { + return nil, err + } + + if err = e.writeToBuffer(buf, headerBuf.Bytes()); err != nil { + return nil, err + } + + // Write the combinedKey to the buffer + if err = binary.Write(buf, binary.LittleEndian, []rune(combinedKey)); err != nil { + return nil, err + } + + // Write the body to the buffer + if err = e.writeToBuffer(buf, body); err != nil { + return nil, err + } + + return buf.Bytes(), nil +} + +// encodeDeletionMessage encodes the given deletionMessage to a binary format. +func (e *encoder) encodeDeletionMessage(message *deletionMessage) ([]byte, error) { + buf := new(bytes.Buffer) + + // calculate the checksum of the deletion message + checksum := calculateChecksum([]byte(fmt.Sprintf("%d:%d:%s:%s", message.St, message.Et, message.Slot, message.Key))) + cMessageHeader := deletionMessageHeaderPreamble{ + St: message.St, + Et: message.Et, + SLen: int16(len(message.Slot)), + KLen: int32(len(message.Key)), + Checksum: checksum, + } + + // write the compact header + if err := binary.Write(buf, binary.LittleEndian, cMessageHeader); err != nil { + return nil, err + } + + // write the slot + if err := binary.Write(buf, binary.LittleEndian, []rune(message.Slot)); err != nil { + return nil, err + } + + // write the key + if err := binary.Write(buf, binary.LittleEndian, []rune(message.Key)); err != nil { + return nil, err + } + + return buf.Bytes(), nil +} + +func calculateChecksum(data []byte) uint32 { + crc32q := crc32.MakeTable(_IEEE) + return crc32.Checksum(data, crc32q) +} + +// encodeWALMessageHeader encodes the WALMessage header. +func (e *encoder) encodeWALMessageHeader(message *isb.ReadMessage, bodyLen int64, checksum uint32, keyLen int32) (*bytes.Buffer, error) { + buf := new(bytes.Buffer) + + offset, err := message.ReadOffset.Sequence() + if err != nil { + log.Println("error getting sequence from offset", err.Error()) + return nil, err + } + + // Prepare the header + hp := &readMessageHeaderPreamble{ + Offset: offset, + WaterMark: message.Watermark.UnixMilli(), + EventTime: message.EventTime.UnixMilli(), + MessageLen: bodyLen, + Checksum: checksum, + KeyLen: keyLen, + } + + // write the fixed values + if err = binary.Write(buf, binary.LittleEndian, hp); err != nil { + return nil, err + } + + return buf, nil +} + +// encodeWALMessageBody uses ReadMessage.Message field as the body of the unalignedWAL message, encodes the +// ReadMessage.Message, and returns. +func (e *encoder) encodeWALMessageBody(readMsg *isb.ReadMessage) ([]byte, error) { + msgBinary, err := readMsg.Message.MarshalBinary() + if err != nil { + return nil, fmt.Errorf("encodeWALMessageBody encountered encode err: %w", err) + } + return msgBinary, nil +} + +func (e *encoder) writeToBuffer(buf *bytes.Buffer, data []byte) error { + wrote, err := buf.Write(data) + if err != nil { + return err + } + expectedLength := len(data) + if wrote != expectedLength { + return fmt.Errorf("expected to write %d, but wrote only %d", expectedLength, wrote) + } + return nil +} diff --git a/pkg/reduce/pbq/wal/unaligned/fs/encoder_test.go b/pkg/reduce/pbq/wal/unaligned/fs/encoder_test.go new file mode 100644 index 000000000..b75911f34 --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/encoder_test.go @@ -0,0 +1,88 @@ +package fs + +import ( + "os" + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/numaproj/numaflow/pkg/isb/testutils" + "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" +) + +// tests for encoder and decoder +func TestEncodingAndDecoding(t *testing.T) { + tempDir := t.TempDir() + defer func() { + cleanupDir(tempDir) + }() + + // create a file to write the encoded messages to + fp, err := os.OpenFile(tempDir+"/testFile", os.O_CREATE|os.O_WRONLY, 0644) + assert.NoError(t, err) + + ec := newEncoder() + if ec == nil { + t.Errorf("Expected newEncoder() to return a non-nil encoder") + } + + partitionId := &partition.ID{ + Start: time.UnixMilli(60000).In(location), + End: time.UnixMilli(70000).In(location), + Slot: "testSlot", + } + + bytes, err := ec.encodeHeader(partitionId) + assert.NoError(t, err) + + // write the header + _, err = fp.Write(bytes) + assert.NoError(t, err) + + // build test read messages + readMessages := testutils.BuildTestReadMessages(100, time.UnixMilli(60000), []string{"key1:key2"}) + for _, msg := range readMessages { + bytes, err = ec.encodeMessage(&msg) + assert.NoError(t, err) + _, err = fp.Write(bytes) + assert.NoError(t, err) + } + + dMsg := &deletionMessage{ + St: 60000, + Et: 120000, + Slot: "testSlot", + Key: "key1:key2", + } + + bytes, err = ec.encodeDeletionMessage(dMsg) + assert.NoError(t, err) + _, err = fp.Write(bytes) + assert.NoError(t, err) + err = fp.Close() + assert.NoError(t, err) + + // create a decoder + dc := newDecoder() + // open the same file in read mode + fp, err = os.OpenFile(tempDir+"/testFile", os.O_RDONLY, 0644) + assert.NoError(t, err) + + pid, err := dc.decodeHeader(fp) + assert.NoError(t, err) + assert.Equal(t, partitionId, pid) + + // decode the messages + for i := 0; i < 100; i++ { + msg, _, err := dc.decodeMessage(fp) + assert.NoError(t, err) + assert.Equal(t, readMessages[i].EventTime.UnixMilli(), msg.EventTime.UnixMilli()) + } + + // decode the deletion message + dm, _, err := dc.decodeDeletionMessage(fp) + assert.NoError(t, err) + assert.Equal(t, dMsg, dm) + +} diff --git a/pkg/reduce/pbq/wal/unaligned/fs/gcevents_wal.go b/pkg/reduce/pbq/wal/unaligned/fs/gcevents_wal.go new file mode 100644 index 000000000..c7ca86207 --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/gcevents_wal.go @@ -0,0 +1,231 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs + +import ( + "bufio" + "context" + "encoding/binary" + "fmt" + "os" + "path/filepath" + "strings" + "time" + + "go.uber.org/zap" + + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/unaligned" + "github.com/numaproj/numaflow/pkg/shared/logging" + "github.com/numaproj/numaflow/pkg/window" +) + +const ( + eventsFilePrefix = "events" + currentEventsFile = "current" + "-" + eventsFilePrefix +) + +type gcEventsWAL struct { + eventsPath string // dir path to the events file + currEventsFile *os.File // current events file to write to + eventsBufWriter *bufio.Writer // buffer writer for the events file + prevSyncedTime time.Time // previous synced time + syncDuration time.Duration // sync duration + encoder *encoder // encoder for the events file + rotationDuration time.Duration // rotation duration + rotationEventsCount int // rotation events count + curEventsCount int // current events count + fileCreationTime time.Time // file creation time + latestEndTime time.Time // latest end time of the window + log *zap.SugaredLogger +} + +// NewGCEventsWAL returns a new GCEventsWAL +func NewGCEventsWAL(ctx context.Context, opts ...GCEventsWALOption) (unaligned.GCEventsWAL, error) { + gw := &gcEventsWAL{ + syncDuration: dfv1.DefaultGCEventsWALSyncDuration, + rotationDuration: dfv1.DefaultGCEventsWALRotationDuration, + eventsPath: dfv1.DefaultGCEventsWALEventsPath, + currEventsFile: nil, + eventsBufWriter: nil, + prevSyncedTime: time.Now(), + encoder: newEncoder(), + rotationEventsCount: dfv1.DefaultGCEventsWALRotationEventsCount, + curEventsCount: 0, + fileCreationTime: time.Now(), + latestEndTime: time.UnixMilli(-1), + log: logging.FromContext(ctx), + } + + for _, opt := range opts { + opt(gw) + } + + var err error + // Create event dir if not exist + if _, err = os.Stat(gw.eventsPath); os.IsNotExist(err) { + err = os.Mkdir(gw.eventsPath, 0755) + if err != nil { + return nil, err + } + } + + // open the events file + err = gw.openEventsFile() + + return gw, err +} + +// rotateEventsFile rotates the events file and updates the current events file +// with the new file +func (g *gcEventsWAL) rotateEventsFile() error { + defer func() { + g.curEventsCount = 0 + g.fileCreationTime = time.Now() + }() + + var err error + if err = g.flushAndSync(); err != nil { + return err + } + + // close the current file + if err = g.currEventsFile.Close(); err != nil { + return err + } + + newFilePath := g.getEventsFilePath() + // rename the current event file to the new file path + if err = os.Rename(filepath.Join(g.eventsPath, currentEventsFile), newFilePath); err != nil { + return err + } + + g.log.Debugw("Rotated the gc events segment", zap.String("new-events-file", newFilePath)) + return g.openEventsFile() +} + +// getEventsFilePath returns the events file path +func (g *gcEventsWAL) getEventsFilePath() string { + return filepath.Join(g.eventsPath, eventsFilePrefix+"-"+fmt.Sprintf("%d-%d", time.Now().UnixNano(), g.latestEndTime.UnixMilli())) +} + +// openEventsFile opens a new events file to write to +func (g *gcEventsWAL) openEventsFile() error { + g.log.Debugw("Opening a new gc events segment") + eventFilePath := filepath.Join(g.eventsPath, currentEventsFile) + + var err error + if g.currEventsFile, err = os.OpenFile(eventFilePath, os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0644); err != nil { + return err + } + + // reset the data buffer writer + if g.eventsBufWriter == nil { + g.eventsBufWriter = bufio.NewWriter(g.currEventsFile) + } else { + g.eventsBufWriter.Reset(g.currEventsFile) + } + return nil +} + +// PersistGCEvent persists the GC event of the window +func (g *gcEventsWAL) PersistGCEvent(window window.TimedWindow) error { + + if g.currEventsFile == nil { + return fmt.Errorf("events file is not open") + } + + dms := &deletionMessage{ + St: window.StartTime().UnixMilli(), + Et: window.EndTime().UnixMilli(), + Slot: window.Slot(), + Key: strings.Join(window.Keys(), dfv1.KeysDelimitter), + } + + // encode and write the deletion message + dBytes, err := g.encoder.encodeDeletionMessage(dms) + if err != nil { + return err + } + + if err = binary.Write(g.eventsBufWriter, binary.LittleEndian, dBytes); err != nil { + return err + } + + // sync the file if the sync duration is elapsed + if time.Since(g.prevSyncedTime) >= g.syncDuration { + if err = g.flushAndSync(); err != nil { + return err + } + } + + // update the latest end time of the window + if window.EndTime().After(g.latestEndTime) { + g.latestEndTime = window.EndTime() + } + + // if rotation events count is reached, or rotation duration is elapsed + // rotate the events file + g.curEventsCount++ + if g.curEventsCount >= g.rotationEventsCount || time.Since(g.fileCreationTime) >= g.rotationDuration { + if err = g.rotateEventsFile(); err != nil { + return err + } + } + + return nil +} + +func (g *gcEventsWAL) flushAndSync() error { + if err := g.eventsBufWriter.Flush(); err != nil { + return err + } + + g.prevSyncedTime = time.Now() + return g.currEventsFile.Sync() +} + +// Close closes the GCEventsWAL by flushing and syncing the current events file +func (g *gcEventsWAL) Close() error { + g.log.Info("Closing the GC events WAL") + + if err := g.flushAndSync(); err != nil { + return err + } + + if err := g.currEventsFile.Close(); err != nil { + return err + } + + // if no events are written to the current events file, delete the file + // else rename the current events file so that it can be read by the compactor + // during startup + if g.curEventsCount == 0 { + // delete the current events file if no events are written + if err := os.Remove(g.currEventsFile.Name()); err != nil { + return err + } + return nil + } + + // rename the current events file to the events file + if err := os.Rename(filepath.Join(g.eventsPath, currentEventsFile), g.getEventsFilePath()); err != nil { + return err + } + + return nil +} diff --git a/pkg/reduce/pbq/wal/unaligned/fs/gcevents_wal_test.go b/pkg/reduce/pbq/wal/unaligned/fs/gcevents_wal_test.go new file mode 100644 index 000000000..608db0cbc --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/gcevents_wal_test.go @@ -0,0 +1,66 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs + +import ( + "context" + "os" + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/numaproj/numaflow/pkg/window" +) + +func TestGcEventsTracker_TrackGCEvent(t *testing.T) { + ctx := context.Background() + tempDir := t.TempDir() + defer func(path string) { + _ = os.RemoveAll(path) + }(tempDir) + + tracker, err := NewGCEventsWAL(ctx, WithEventsPath(tempDir), WithGCTrackerSyncDuration(100*time.Millisecond), WithGCTrackerRotationDuration(time.Second)) + assert.NoError(t, err) + + // build test windows + ts := time.UnixMilli(60000) + windows := buildTestWindows(ts, 100, time.Second, []string{"key-1", "key-2"}) + for _, timedWindow := range windows { + err = tracker.PersistGCEvent(timedWindow) + time.Sleep(time.Millisecond * 10) + assert.NoError(t, err) + } + + err = tracker.Close() + assert.NoError(t, err) + + // list all the filesToReplay in the directory + files, err := os.ReadDir(tempDir) + assert.NoError(t, err) + assert.NotEmpty(t, files) +} + +func buildTestWindows(ts time.Time, count int, windowSize time.Duration, keys []string) []window.TimedWindow { + var windows = make([]window.TimedWindow, 0, count) + for i := 0; i < count; i++ { + windows = append(windows, window.NewUnalignedTimedWindow(ts, ts.Add(windowSize), "slot-0", keys)) + ts = ts.Add(windowSize) + } + + return windows +} diff --git a/pkg/reduce/pbq/wal/unaligned/fs/manager.go b/pkg/reduce/pbq/wal/unaligned/fs/manager.go new file mode 100644 index 000000000..bfc3cba7b --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/manager.go @@ -0,0 +1,142 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs + +import ( + "context" + "os" + "path/filepath" + + "go.uber.org/zap" + + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" + "github.com/numaproj/numaflow/pkg/shared/logging" +) + +type fsWAL struct { + segmentWALPath string + compactWALPath string + pipelineName string + vertexName string + replicaIndex int32 + fsOpts []WALOption + // we don't need a lock to access activeWALs, since we use only one partition + // when we start using slots, we will need a lock + log *zap.SugaredLogger + activeWALs map[string]wal.WAL +} + +// NewFSManager is a FileSystem Stores Manager. +func NewFSManager(ctx context.Context, segmentWALPath string, compactWALPath string, vertexInstance *dfv1.VertexInstance, opts ...WALOption) wal.Manager { + opts = append(opts, WithSegmentWALPath(segmentWALPath), WithCompactWALPath(compactWALPath)) + + s := &fsWAL{ + segmentWALPath: segmentWALPath, + compactWALPath: compactWALPath, + pipelineName: vertexInstance.Vertex.Spec.PipelineName, + vertexName: vertexInstance.Vertex.Spec.AbstractVertex.Name, + replicaIndex: vertexInstance.Replica, + activeWALs: make(map[string]wal.WAL), + log: logging.FromContext(ctx), + fsOpts: opts, + } + + // create the segment and compact dir if not exist + if _, err := os.Stat(segmentWALPath); os.IsNotExist(err) { + err = os.Mkdir(segmentWALPath, 0755) + if err != nil { + return nil + } + } + + if _, err := os.Stat(compactWALPath); os.IsNotExist(err) { + err = os.Mkdir(compactWALPath, 0755) + if err != nil { + return nil + } + } + return s +} + +// CreateWAL creates the FS unalignedWAL. +func (ws *fsWAL) CreateWAL(ctx context.Context, partitionID partition.ID) (wal.WAL, error) { + // check if the WAL is already present during crash recovery, + // we might have already created the WAL while replaying + if store, ok := ws.activeWALs[partitionID.String()]; ok { + return store, nil + } + + w, err := NewUnalignedWriteOnlyWAL(ctx, &partitionID, ws.fsOpts...) + if err != nil { + return nil, err + } + + ws.activeWALs[w.PartitionID().String()] = w + ws.log.Infow("Created Unaligned WAL", zap.String("partitionID", w.PartitionID().String())) + return w, nil +} + +// DiscoverWALs returns all the WALs present in the segmentWALPath +func (ws *fsWAL) DiscoverWALs(ctx context.Context) ([]wal.WAL, error) { + wr := make([]wal.WAL, 0) + + // check if there are any compacted or segment segmentFiles to replay + compactedFiles, err := listFilesInDir(ws.compactWALPath, currentWALPrefix, sortFunc) + if err != nil { + return nil, err + } + segmentFiles, err := listFilesInDir(ws.segmentWALPath, currentWALPrefix, sortFunc) + if err != nil { + return nil, err + } + + // if there are no segmentFiles to replay, return + if len(segmentFiles) == 0 && len(compactedFiles) == 0 { + return wr, nil + } + + // consider the compacted files for replay first + // since the compacted files are the oldest + ws.log.Infow("Number of files to replay", zap.Int("count", len(segmentFiles)+len(compactedFiles))) + filesToReplay := make([]string, 0) + for _, file := range compactedFiles { + ws.log.Infow("compacted file to replay", zap.String("file", file.Name())) + filesToReplay = append(filesToReplay, filepath.Join(ws.compactWALPath, file.Name())) + } + for _, file := range segmentFiles { + ws.log.Infow("segment file to replay", zap.String("file", file.Name())) + filesToReplay = append(filesToReplay, filepath.Join(ws.segmentWALPath, file.Name())) + } + + // there will only be one WAL because we use shared partition + // for unaligned windows + wl, err := NewUnalignedReadWriteWAL(ctx, filesToReplay, ws.fsOpts...) + if err != nil { + return nil, err + } + + ws.activeWALs[wl.PartitionID().String()] = wl + return append(wr, wl), nil +} + +// DeleteWAL deletes the store for the given partitionID +func (ws *fsWAL) DeleteWAL(partitionID partition.ID) error { + delete(ws.activeWALs, partitionID.String()) + return nil +} diff --git a/pkg/reduce/pbq/wal/unaligned/fs/options.go b/pkg/reduce/pbq/wal/unaligned/fs/options.go new file mode 100644 index 000000000..443252642 --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/options.go @@ -0,0 +1,118 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs + +import ( + "time" +) + +type WALOption func(stores *unalignedWAL) + +// WithSegmentWALPath sets the segment WAL path +func WithSegmentWALPath(path string) WALOption { + return func(stores *unalignedWAL) { + stores.segmentWALPath = path + } +} + +// WithCompactWALPath sets the compact WAL path +func WithCompactWALPath(path string) WALOption { + return func(stores *unalignedWAL) { + stores.compactWALPath = path + } +} + +// WithSyncDuration sets the sync duration +func WithSyncDuration(maxDuration time.Duration) WALOption { + return func(stores *unalignedWAL) { + stores.syncDuration = maxDuration + } +} + +// WithMaxBatchSize sets the max batch size +func WithMaxBatchSize(size int64) WALOption { + return func(stores *unalignedWAL) { + stores.maxBatchSize = size + } +} + +// WithSegmentRotationDuration sets the segment rotation duration +func WithSegmentRotationDuration(maxDuration time.Duration) WALOption { + return func(stores *unalignedWAL) { + stores.segmentRotationDuration = maxDuration + } +} + +// WithSegmentSize sets the segment size +func WithSegmentSize(size int64) WALOption { + return func(stores *unalignedWAL) { + stores.segmentSize = size + } +} + +type GCEventsWALOption func(tracker *gcEventsWAL) + +// WithGCTrackerRotationDuration sets the rotation duration for the GC events WAL +func WithGCTrackerRotationDuration(rotationDuration time.Duration) GCEventsWALOption { + return func(tracker *gcEventsWAL) { + tracker.rotationDuration = rotationDuration + } +} + +// WithEventsPath sets the path for the GC events WAL +func WithEventsPath(path string) GCEventsWALOption { + return func(tracker *gcEventsWAL) { + tracker.eventsPath = path + } +} + +// WithGCTrackerSyncDuration sets the sync duration for the GC events WAL +func WithGCTrackerSyncDuration(maxDuration time.Duration) GCEventsWALOption { + return func(tracker *gcEventsWAL) { + tracker.syncDuration = maxDuration + } +} + +// WithGCTrackerRotationEventsCount sets the rotation events count for the GC events WAL +func WithGCTrackerRotationEventsCount(count int) GCEventsWALOption { + return func(tracker *gcEventsWAL) { + tracker.rotationEventsCount = count + } +} + +type CompactorOption func(c *compactor) + +// WithCompactorMaxFileSize sets the max file size for the compactor +func WithCompactorMaxFileSize(maxFileSize int64) CompactorOption { + return func(c *compactor) { + c.maxFileSize = maxFileSize + } +} + +// WithCompactorSyncDuration sets the sync duration for the compactor +func WithCompactorSyncDuration(maxDuration time.Duration) CompactorOption { + return func(c *compactor) { + c.syncDuration = maxDuration + } +} + +// WithCompactionDuration sets the compaction duration for the compactor +func WithCompactionDuration(maxDuration time.Duration) CompactorOption { + return func(c *compactor) { + c.compactionDuration = maxDuration + } +} diff --git a/pkg/reduce/pbq/wal/unaligned/fs/wal_benchmark_test.go b/pkg/reduce/pbq/wal/unaligned/fs/wal_benchmark_test.go new file mode 100644 index 000000000..8feb4dd7d --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/wal_benchmark_test.go @@ -0,0 +1,56 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs + +import ( + "bufio" + "fmt" + "os" + "testing" +) + +const output = "this is just an example string%\n" + +func BenchmarkBufioWriter(b *testing.B) { + file, _ := os.Create("test.txt") + defer func(file *os.File) { + _ = file.Close() + _ = os.Remove("test.txt") + }(file) + buffer := bufio.NewWriter(file) + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + _, _ = buffer.Write([]byte(output + fmt.Sprintf("%d\n", i))) + } + _ = buffer.Flush() +} + +func BenchmarkFileWriteAt(b *testing.B) { + file, _ := os.Create("test.txt") + defer func(file *os.File) { + _ = file.Close() + _ = os.Remove("test.txt") + }(file) + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + _, _ = file.WriteAt([]byte(output+fmt.Sprintf("%d\n", i)), int64(i)) + } +} diff --git a/pkg/reduce/pbq/wal/unaligned/fs/wal_segment.go b/pkg/reduce/pbq/wal/unaligned/fs/wal_segment.go new file mode 100644 index 000000000..eacd93d0f --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/wal_segment.go @@ -0,0 +1,390 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs + +import ( + "bufio" + "context" + "fmt" + "io" + "os" + "path/filepath" + "time" + + "go.uber.org/zap" + + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/isb" + "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal" + "github.com/numaproj/numaflow/pkg/shared/logging" +) + +const ( + currentWALPrefix = "current" + segmentPrefix = "segment" + currentSegmentName = currentWALPrefix + "-" + segmentPrefix +) + +// unalignedWAL is an unaligned write-ahead log +type unalignedWAL struct { + partitionID *partition.ID // partitionID is the partition ID for the unalignedWAL + currDataFp *os.File // currDataFp is the current data file pointer to which the data is being written + currWriteOffset int64 // currWriteOffset is the current write offset + prevSyncedWOffset int64 // prevSyncedWOffset is the previous synced write offset + dataBufWriter *bufio.Writer // dataBufWriter is the buffered writer for the data file + prevSyncedTime time.Time // prevSyncedTime is the previous synced time + segmentCreateTime time.Time // segmentCreateTime is the time when the segment is created + encoder *encoder // encoder is the encoder for the unalignedWAL entries and header + decoder *decoder // decoder is the decoder for the unalignedWAL entries and header + segmentWALPath string // segmentWALPath is the path to the unalignedWAL data + compactWALPath string // compactWALPath is the path to the compacted unalignedWAL data + segmentSize int64 // segmentSize is the max size of the segment + syncDuration time.Duration // syncDuration is the duration after which the data is synced to the disk + maxBatchSize int64 // maxBatchSize is the maximum batch size before the data is synced to the disk + segmentRotationDuration time.Duration // segmentRotationDuration is the duration after which the segment is rotated + filesToReplay []string + latestWm time.Time + log *zap.SugaredLogger +} + +// NewUnalignedWriteOnlyWAL returns a new store writer instance +func NewUnalignedWriteOnlyWAL(ctx context.Context, partitionId *partition.ID, opts ...WALOption) (wal.WAL, error) { + + s := &unalignedWAL{ + segmentWALPath: dfv1.DefaultSegmentWALPath, + segmentSize: dfv1.DefaultWALSegmentSize, + maxBatchSize: dfv1.DefaultWALMaxSyncSize, + syncDuration: dfv1.DefaultWALSyncDuration, + segmentRotationDuration: dfv1.DefaultWALSegmentRotationDuration, + currDataFp: nil, + dataBufWriter: nil, + currWriteOffset: 0, + prevSyncedWOffset: 0, + prevSyncedTime: time.Now(), + segmentCreateTime: time.Now(), + encoder: newEncoder(), + decoder: newDecoder(), + partitionID: partitionId, + log: logging.FromContext(ctx), + } + + for _, opt := range opts { + opt(s) + } + + // open the current data file + err := s.openFileAndSetCurrent() + if err != nil { + return nil, err + } + return s, nil +} + +// NewUnalignedReadWriteWAL returns a new WAL instance for reading and writing +func NewUnalignedReadWriteWAL(ctx context.Context, filesToReplay []string, opts ...WALOption) (wal.WAL, error) { + + s := &unalignedWAL{ + segmentWALPath: dfv1.DefaultSegmentWALPath, + segmentSize: dfv1.DefaultWALSegmentSize, + maxBatchSize: dfv1.DefaultWALMaxSyncSize, + syncDuration: dfv1.DefaultWALSyncDuration, + segmentRotationDuration: dfv1.DefaultWALSegmentRotationDuration, + currDataFp: nil, + dataBufWriter: nil, + currWriteOffset: 0, + prevSyncedWOffset: 0, + prevSyncedTime: time.Now(), + segmentCreateTime: time.Now(), + encoder: newEncoder(), + decoder: newDecoder(), + filesToReplay: filesToReplay, + latestWm: time.UnixMilli(-1), + log: logging.FromContext(ctx), + } + + for _, opt := range opts { + opt(s) + } + + // open a file to get the partition ID + fp, pid, err := s.openReadFile(filesToReplay[0]) + if err != nil { + return nil, err + } + + // set the partition ID to the WAL + s.partitionID = pid + + // close the file + _ = fp.Close() + + return s, nil +} + +// Write writes the message to the unalignedWAL. The format as follow is +// +// +--------------------+-------------------+-----------------+------------------+-----------------+-------------+------------+----------------+ +// | event time (int64) | watermark (int64) | offset (int64) | msg-len (int64) | key-len (int64) | CRC (uint32 | key []byte | message []byte | +// +--------------------+-------------------+-----------------+------------------+-----------------+-------------+------------+----------------+ +// +// CRC will be used for detecting ReadMessage corruptions. +func (s *unalignedWAL) Write(message *isb.ReadMessage) error { + + // encode the message + entry, err := s.encoder.encodeMessage(message) + if err != nil { + return err + } + + // write the message to the data file + wrote, err := s.dataBufWriter.Write(entry) + if wrote != len(entry) { + return fmt.Errorf("expected to write %d, but wrote only %d, %w", len(entry), wrote, err) + } + if err != nil { + return err + } + + currTime := time.Now() + + // sync file if the batch size is reached or sync duration is reached + if s.currWriteOffset-s.prevSyncedWOffset >= s.maxBatchSize || currTime.Sub(s.prevSyncedTime) >= s.syncDuration { + if err = s.flushAndSync(); err != nil { + return err + } + } + + // only increase the offset when we successfully write for atomicity. + s.currWriteOffset += int64(wrote) + + // update the watermark if its not -1 + // we could have done a comparison check but this is more efficient as we are not comparing complex + // time values. + // this watermark is monotonically increasing but -1 is an accepted value. + // the below condition becoming true is the likely path (i.e., we will always update s.latestWm) + if message.Watermark.UnixMilli() != -1 { + s.latestWm = message.Watermark + } + + // rotate the segment if the segment size is reached or segment duration is reached + if currTime.Sub(s.segmentCreateTime) >= s.segmentRotationDuration || s.currWriteOffset >= s.segmentSize { + if err = s.rotateFile(); err != nil { + return err + } + } + + return nil +} + +// Replay replays persisted messages during startup +// It returns a channel to read messages from replay files and a channel to read errors +func (s *unalignedWAL) Replay() (<-chan *isb.ReadMessage, <-chan error) { + // Initialize channels + msgChan := make(chan *isb.ReadMessage) + errChan := make(chan error) + + go func() { + // Clean up resources when the function returns + defer close(msgChan) + defer func() { errChan = nil }() + + // Iterate over all replay files + for _, filePath := range s.filesToReplay { + // Open the file in read mode + fp, _, err := s.openReadFile(filePath) + if err != nil { + errChan <- err + return + } + + // Iterate over the messages in the file + for { + // Try to decode a message from the file + msg, _, err := s.decoder.decodeMessage(fp) + if err == io.EOF { + // End of file reached, break the inner loop to proceed to the next file + break + } else if err != nil { + // Error occurred, send it on error channel and return + errChan <- err + return + } + + // Successful decode, send the message on the message channel + msgChan <- msg + } + + // Close the file + err = fp.Close() + if err != nil { + errChan <- err + return + } + } + + // Open a new write file once replay is done since we use the same WAL for reading and writing + err := s.openFileAndSetCurrent() + if err != nil { + errChan <- err + return + } + + s.filesToReplay = nil + }() + + // Return the message and error channels + return msgChan, errChan +} + +// PartitionID returns the partition ID of the store +func (s *unalignedWAL) PartitionID() *partition.ID { + return s.partitionID +} + +func (s *unalignedWAL) openReadFile(filePath string) (*os.File, *partition.ID, error) { + + // Open the first file in the list + currFile, err := os.OpenFile(filePath, os.O_RDONLY, 0644) + if err != nil { + return nil, nil, err + } + + pid, err := s.decoder.decodeHeader(currFile) + if err != nil { + return nil, nil, err + } + + return currFile, pid, nil +} + +// openFileAndSetCurrent opens a new data file and sets the current pointer to the opened file. +func (s *unalignedWAL) openFileAndSetCurrent() error { + s.log.Infow("Opening new segment file") + + dataFilePath := filepath.Join(s.segmentWALPath, currentSegmentName) + var err error + if s.currDataFp, err = os.OpenFile(dataFilePath, os.O_WRONLY|os.O_CREATE, 0644); err != nil { + return err + } + + // reset the data buffer writer + if s.dataBufWriter == nil { + s.dataBufWriter = bufio.NewWriter(s.currDataFp) + } else { + s.dataBufWriter.Reset(s.currDataFp) + } + + // reset the offset + s.currWriteOffset = 0 + + // write the WAL header to the new file + return s.writeWALHeader() +} + +// rotateFile rotates the current data file to the segment file +// and updates the current data file to a new file. +func (s *unalignedWAL) rotateFile() error { + defer func() { + s.segmentCreateTime = time.Now() + }() + // check if there are any messages written to the file + // if not, we don't need to rotate the file + // can happen when there are no messages to write + // within the rotation duration + if s.currWriteOffset == 0 { + return nil + } + + // Sync data before rotating the file + if err := s.flushAndSync(); err != nil { + return err + } + + // Close the current data file + if err := s.currDataFp.Close(); err != nil { + return err + } + + newFileName := s.segmentFilePath(s.segmentWALPath) + + // rename the current data file to the segment file + if err := os.Rename(filepath.Join(s.segmentWALPath, currentSegmentName), newFileName); err != nil { + return err + } + + s.log.Debugw("Rotated segment file to - ", zap.String("fileName", newFileName)) + + // Open the next data file + return s.openFileAndSetCurrent() +} + +// segmentFilePath creates the file path for the segment file located in the storage path. +func (s *unalignedWAL) segmentFilePath(storePath string) string { + return filepath.Join(storePath, segmentPrefix+"-"+fmt.Sprintf("%d-%d", time.Now().UnixNano(), s.latestWm.UnixMilli())) +} + +// flushAndSync flushes the buffered data to the writer and syncs the file to disk. +func (s *unalignedWAL) flushAndSync() error { + if err := s.dataBufWriter.Flush(); err != nil { + return err + } + + if err := s.currDataFp.Sync(); err != nil { + return err + } + + s.prevSyncedWOffset = s.currWriteOffset + s.prevSyncedTime = time.Now() + + return nil +} + +// Close closes unalignedWAL +func (s *unalignedWAL) Close() error { + // sync data before closing + err := s.flushAndSync() + if err != nil { + return err + } + + // close the current data segment + if err = s.currDataFp.Close(); err != nil { + return err + } + + // rename the current data file to the segment file + if err = os.Rename(filepath.Join(s.segmentWALPath, currentSegmentName), s.segmentFilePath(s.segmentWALPath)); err != nil { + return err + } + + return nil +} + +// writeWALHeader writes the unalignedWAL header to the file. +func (s *unalignedWAL) writeWALHeader() error { + header, err := s.encoder.encodeHeader(s.partitionID) + if err != nil { + return err + } + wrote, err := s.dataBufWriter.Write(header) + if wrote != len(header) { + return fmt.Errorf("expected to write %d, but wrote only %d, %w", len(header), wrote, err) + } + + return err +} diff --git a/pkg/reduce/pbq/wal/unaligned/fs/wal_segment_test.go b/pkg/reduce/pbq/wal/unaligned/fs/wal_segment_test.go new file mode 100644 index 000000000..9cd30e36c --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/fs/wal_segment_test.go @@ -0,0 +1,148 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 fs + +import ( + "context" + "os" + "testing" + "time" + + "github.com/stretchr/testify/assert" + + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/isb" + "github.com/numaproj/numaflow/pkg/isb/testutils" + "github.com/numaproj/numaflow/pkg/window" +) + +var vertexInstance = &dfv1.VertexInstance{ + Vertex: &dfv1.Vertex{Spec: dfv1.VertexSpec{ + PipelineName: "testPipeline", + AbstractVertex: dfv1.AbstractVertex{ + Name: "testVertex", + Sink: &dfv1.Sink{ + Kafka: &dfv1.KafkaSink{}, + }, + }, + }}, + Replica: 0, +} + +func TestUnalignedWAL_Write(t *testing.T) { + + ctx := context.Background() + segmentDir := t.TempDir() + defer func(path string) { + cleanupDir(path) + }(segmentDir) + + compactDir := t.TempDir() + defer func(path string) { + cleanupDir(path) + }(compactDir) + + partitionId := window.SharedUnalignedPartition + s, err := NewUnalignedWriteOnlyWAL(ctx, &partitionId, WithStoreOptions(segmentDir, compactDir)) + assert.NoError(t, err) + + // create read messages + readMessages := testutils.BuildTestReadMessagesIntOffset(100, time.UnixMilli(60000), nil) + + // write the messages + for _, readMessage := range readMessages { + err = s.Write(&readMessage) + assert.NoError(t, err) + } + + // close the unalignedWAL + err = s.Close() + assert.NoError(t, err) + + // list all the filesToReplay in the directory + files, err := os.ReadDir(segmentDir) + assert.NoError(t, err) + assert.NotEmpty(t, files) +} + +func TestUnalignedWAL_Replay(t *testing.T) { + ctx := context.Background() + tempDir := t.TempDir() + defer func(path string) { + cleanupDir(path) + }(tempDir) + + compactDir := t.TempDir() + defer func(path string) { + cleanupDir(path) + }(compactDir) + + partitionId := window.SharedUnalignedPartition + s, err := NewUnalignedWriteOnlyWAL(ctx, &partitionId, WithStoreOptions(tempDir, compactDir)) + assert.NoError(t, err) + + // create read messages + readMessages := testutils.BuildTestReadMessagesIntOffset(1000, time.UnixMilli(60000), []string{"key-1", "key-2"}) + + // write the messages + for _, readMessage := range readMessages { + err = s.Write(&readMessage) + assert.NoError(t, err) + } + + // close the unalignedWAL + err = s.Close() + assert.NoError(t, err) + + wm := NewFSManager(ctx, tempDir, compactDir, vertexInstance) + wls, err := wm.DiscoverWALs(ctx) + assert.NoError(t, err) + + // replay the messages + readCh, errCh := wls[0].Replay() + replayedMessages := make([]*isb.ReadMessage, 0) +readLoop: + for { + select { + case msg, ok := <-readCh: + if !ok { + break readLoop + } + replayedMessages = append(replayedMessages, msg) + case err := <-errCh: + assert.NoError(t, err) + } + } + assert.NoError(t, err) + assert.Equal(t, len(readMessages), len(replayedMessages)) + + // order is important + for i := 0; i < len(readMessages); i++ { + assert.Equal(t, readMessages[i].EventTime.UnixMilli(), replayedMessages[i].EventTime.UnixMilli()) + } +} + +func WithStoreOptions(segmentPath string, compactPath string) WALOption { + return func(s *unalignedWAL) { + s.segmentWALPath = segmentPath + s.compactWALPath = compactPath + s.segmentSize = 1024 * 1024 * 10 + s.syncDuration = 10 * time.Second + s.maxBatchSize = 1024 * 50 + s.segmentRotationDuration = 120 * time.Second + } +} diff --git a/pkg/reduce/pbq/wal/unaligned/interfaces.go b/pkg/reduce/pbq/wal/unaligned/interfaces.go new file mode 100644 index 000000000..edfbc4d3f --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/interfaces.go @@ -0,0 +1,40 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 unaligned + +import ( + "context" + + "github.com/numaproj/numaflow/pkg/window" +) + +// Compactor compacts the unalignedWAL by deleting the persisted messages +// which belongs to the materialized window. +type Compactor interface { + // Start starts the compactor + Start(ctx context.Context) error + // Stop stops the compactor + Stop() error +} + +// GCEventsWAL persists the GC events from PnF of unaligned windows. +type GCEventsWAL interface { + // PersistGCEvent persists the GC event of the window + PersistGCEvent(window window.TimedWindow) error + // Close closes the GCEventsWAL + Close() error +} diff --git a/pkg/reduce/pbq/wal/unaligned/noop/noop.go b/pkg/reduce/pbq/wal/unaligned/noop/noop.go new file mode 100644 index 000000000..bb3071249 --- /dev/null +++ b/pkg/reduce/pbq/wal/unaligned/noop/noop.go @@ -0,0 +1,42 @@ +package noop + +import ( + "context" + + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/unaligned" + "github.com/numaproj/numaflow/pkg/window" +) + +// noopCompactor is a no-op compactor which does not do any operation but can be safely invoked. +type noopCompactor struct { +} + +// NewNoopCompactor returns a new no-op compactor +func NewNoopCompactor() unaligned.Compactor { + return &noopCompactor{} +} + +func (n noopCompactor) Start(ctx context.Context) error { + return nil +} + +func (n noopCompactor) Stop() error { + return nil +} + +// noopGCEventsWAL is a no-op gc events WAL which does not do any operation but can be safely invoked. +type noopGCEventsWAL struct { +} + +// NewNoopGCEventsWAL returns a new no-op GCEventsWAL +func NewNoopGCEventsWAL() unaligned.GCEventsWAL { + return &noopGCEventsWAL{} +} + +func (n noopGCEventsWAL) PersistGCEvent(window window.TimedWindow) error { + return nil +} + +func (n noopGCEventsWAL) Close() error { + return nil +} diff --git a/pkg/reduce/pnf/options.go b/pkg/reduce/pnf/options.go new file mode 100644 index 000000000..ca6cb8867 --- /dev/null +++ b/pkg/reduce/pnf/options.go @@ -0,0 +1,45 @@ +/* +Copyright 2022 The Numaproj Authors. + +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 pnf + +import ( + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/unaligned" + "github.com/numaproj/numaflow/pkg/window" +) + +type options struct { + gcEventsTracker unaligned.GCEventsWAL + windowType window.Type +} + +type Option func(options *options) error + +// WithGCEventsTracker sets the GCEventsWAL option +func WithGCEventsTracker(gcTracker unaligned.GCEventsWAL) Option { + return func(o *options) error { + o.gcEventsTracker = gcTracker + return nil + } +} + +// WithWindowType sets the window type option +func WithWindowType(windowType window.Type) Option { + return func(o *options) error { + o.windowType = windowType + return nil + } +} diff --git a/pkg/reduce/pnf/pnfmanager.go b/pkg/reduce/pnf/pnfmanager.go index e0592ccf0..f9cdfd062 100644 --- a/pkg/reduce/pnf/pnfmanager.go +++ b/pkg/reduce/pnf/pnfmanager.go @@ -47,6 +47,7 @@ type Manager struct { idleManager wmb.IdleManager windower window.TimedWindower pnfRoutines []*processAndForward + opts []Option log *zap.SugaredLogger sync.RWMutex } @@ -60,7 +61,8 @@ func NewPnFManager(ctx context.Context, whereToDecider forwarder.ToWhichStepDecider, watermarkPublishers map[string]publish.Publisher, idleManager wmb.IdleManager, - windower window.TimedWindower) *Manager { + windower window.TimedWindower, + opts ...Option) *Manager { of := &Manager{ vertexName: vertexInstance.Vertex.Spec.Name, @@ -75,6 +77,7 @@ func NewPnFManager(ctx context.Context, windower: windower, pnfRoutines: make([]*processAndForward, 0), log: logging.FromContext(ctx), + opts: opts, } return of @@ -86,7 +89,7 @@ func (op *Manager) AsyncSchedulePnF(ctx context.Context, partitionID *partition.ID, pbq pbq.Reader, ) { - pf := newProcessAndForward(ctx, op.vertexName, op.pipelineName, op.vertexReplica, partitionID, op.reduceApplier, pbq, op.toBuffers, op.whereToDecider, op.watermarkPublishers, op.idleManager, op.pbqManager, op.windower) + pf := newProcessAndForward(ctx, op.vertexName, op.pipelineName, op.vertexReplica, partitionID, op.reduceApplier, pbq, op.toBuffers, op.whereToDecider, op.watermarkPublishers, op.idleManager, op.pbqManager, op.windower, op.opts...) op.pnfRoutines = append(op.pnfRoutines, pf) } diff --git a/pkg/reduce/pnf/processandforward.go b/pkg/reduce/pnf/processandforward.go index 980a68ea3..e85260905 100644 --- a/pkg/reduce/pnf/processandforward.go +++ b/pkg/reduce/pnf/processandforward.go @@ -63,6 +63,7 @@ type processAndForward struct { windower window.TimedWindower latestWriteOffsets map[string][][]isb.Offset done chan struct{} + opts *options } // newProcessAndForward will return a new processAndForward instance @@ -78,7 +79,16 @@ func newProcessAndForward(ctx context.Context, pw map[string]publish.Publisher, idleManager wmb.IdleManager, manager *pbq.Manager, - windower window.TimedWindower) *processAndForward { + windower window.TimedWindower, + opts ...Option) *processAndForward { + + // apply the options + dOpts := &options{} + for _, opt := range opts { + if err := opt(dOpts); err != nil { + logging.FromContext(ctx).Panic("Got an error while applying options", zap.Error(err)) + } + } // latestWriteOffsets tracks the latest write offsets for each ISB buffer. // which will be used for publishing the watermark when the window is closed. @@ -103,6 +113,7 @@ func newProcessAndForward(ctx context.Context, done: make(chan struct{}), latestWriteOffsets: latestWriteOffsets, log: logging.FromContext(ctx), + opts: dOpts, } // start the processAndForward routine. This go-routine is collected by the Shutdown method which // listens on the done channel. @@ -134,6 +145,7 @@ outerLoop: select { case err := <-errCh: if errors.Is(err, context.Canceled) { + p.log.Infow("Context is canceled, stopping the processAndForward", zap.Error(err), zap.Any("partitionID", p.partitionId)) return } if err != nil { @@ -148,13 +160,18 @@ outerLoop: if response.EOF { // since we track session window for every key, we need to delete the closed windows // when we have received the EOF response from the UDF. - // FIXME(session): we need to compact the pbq for unAligned when we have received the EOF response from the UDF. // we should not use p.partitionId here, we should use the partition id from the response. // because for unaligned p.partitionId indicates the shared partition id for the key. p.publishWM(ctx, response.Window.EndTime()) // delete the closed windows which are tracked by the windower p.windower.DeleteClosedWindow(response.Window) + + // FIXME: retry + err := p.opts.gcEventsTracker.PersistGCEvent(response.Window) + if err != nil { + p.log.Errorw("Got an error while tracking GC event", zap.Error(err), zap.Any("partitionID", p.partitionId)) + } continue } @@ -353,7 +370,6 @@ func (p *processAndForward) writeToBuffer(ctx context.Context, edgeName string, } // publishWM publishes the watermark to each edge. -// TODO: support multi partitioned edges. func (p *processAndForward) publishWM(ctx context.Context, endTime time.Time) { // publish watermark, we publish window end time minus one millisecond as watermark // but if there's a window that's about to be closed which has a end time before the current window end time, diff --git a/pkg/reduce/pnf/processandforward_test.go b/pkg/reduce/pnf/processandforward_test.go index c5bd235db..d7f15e607 100644 --- a/pkg/reduce/pnf/processandforward_test.go +++ b/pkg/reduce/pnf/processandforward_test.go @@ -31,7 +31,7 @@ import ( "github.com/numaproj/numaflow/pkg/isb/testutils" "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store/aligned/memory" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/aligned/memory" "github.com/numaproj/numaflow/pkg/shared/kvs" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/watermark/entity" @@ -116,12 +116,12 @@ func TestWriteToBuffer(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second) defer cancel() var pbqManager *pbq.Manager - pbqManager, _ = pbq.NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(), window.Aligned) + pbqManager, _ = pbq.NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemManager(), window.Aligned) toBuffer := map[string][]isb.BufferWriter{ "buffer": value.buffers, } pf, _ := createProcessAndForwardAndOTStore(ctx, value.name, pbqManager, toBuffer) - windowResponse := testutils.BuildTestWriteMessages(int64(15), testStartTime) + windowResponse := testutils.BuildTestWriteMessages(int64(15), testStartTime, nil) pf.writeToBuffer(ctx, "buffer", 0, windowResponse) }) } @@ -200,11 +200,6 @@ func TestPnFHandleAlignedWindowResponses(t *testing.T) { wg.Wait() - for _, writeOffsets := range latestWriteOffsets { - for _, offsets := range writeOffsets { - println(offsets[0].Sequence()) - } - } assert.Equal(t, true, test1Buffer11.IsFull()) assert.Equal(t, true, test1Buffer12.IsFull()) assert.Equal(t, true, test1Buffer21.IsFull()) diff --git a/pkg/sdkclient/reducer/client.go b/pkg/sdkclient/reducer/client.go index 9946b2a6b..92af33ea2 100644 --- a/pkg/sdkclient/reducer/client.go +++ b/pkg/sdkclient/reducer/client.go @@ -18,6 +18,7 @@ package reducer import ( "context" + "errors" "io" reducepb "github.com/numaproj/numaflow-go/pkg/apis/proto/reduce/v1" @@ -86,9 +87,9 @@ func (c *client) ReduceFn(ctx context.Context, datumStreamCh <-chan *reducepb.Re stream, err := c.grpcClt.ReduceFn(ctx) if err != nil { - go func() { - errCh <- util.ToUDFErr("c.grpcClt.ReduceFn", err) - }() + go func(sErr error) { + errCh <- util.ToUDFErr("c.grpcClt.ReduceFn", sErr) + }(err) } // read from the datumStreamCh channel and send it to the server stream @@ -103,14 +104,16 @@ func (c *client) ReduceFn(ctx context.Context, datumStreamCh <-chan *reducepb.Re if !ok { break outerLoop } - if sendErr = stream.Send(datum); sendErr != nil { + // TODO: figure out why send is getting EOF (could be because the client has already handled SIGTERM) + if sendErr = stream.Send(datum); sendErr != nil && !errors.Is(sendErr, io.EOF) { errCh <- util.ToUDFErr("ReduceFn stream.Send()", sendErr) + return } } } + // close the stream after sending all the messages - sendErr = stream.CloseSend() - if sendErr != nil { + if sendErr = stream.CloseSend(); sendErr != nil && !errors.Is(sendErr, io.EOF) { errCh <- util.ToUDFErr("ReduceFn stream.Send()", sendErr) } }() @@ -128,7 +131,7 @@ func (c *client) ReduceFn(ctx context.Context, datumStreamCh <-chan *reducepb.Re default: resp, recvErr = stream.Recv() // if the stream is closed, close the responseCh return - if recvErr == io.EOF { + if errors.Is(recvErr, io.EOF) { // nil channel will never be selected errCh = nil close(responseCh) diff --git a/pkg/sdkclient/sessionreducer/client.go b/pkg/sdkclient/sessionreducer/client.go index 489cb8797..0d59bda8b 100644 --- a/pkg/sdkclient/sessionreducer/client.go +++ b/pkg/sdkclient/sessionreducer/client.go @@ -18,6 +18,7 @@ package sessionreducer import ( "context" + "errors" "io" sessionreducepb "github.com/numaproj/numaflow-go/pkg/apis/proto/sessionreduce/v1" @@ -103,14 +104,14 @@ func (c *client) SessionReduceFn(ctx context.Context, datumStreamCh <-chan *sess if !ok { break outerLoop } - if sendErr = stream.Send(datum); sendErr != nil { + if sendErr = stream.Send(datum); sendErr != nil && !errors.Is(sendErr, io.EOF) { errCh <- util.ToUDFErr("SessionReduceFn stream.Send()", sendErr) + return } } } // close the stream after sending all the messages - sendErr = stream.CloseSend() - if sendErr != nil { + if sendErr = stream.CloseSend(); sendErr != nil && !errors.Is(sendErr, io.EOF) { errCh <- util.ToUDFErr("SessionReduceFn stream.Send()", sendErr) } }() @@ -128,7 +129,7 @@ func (c *client) SessionReduceFn(ctx context.Context, datumStreamCh <-chan *sess default: resp, recvErr = stream.Recv() // if the stream is closed, close the responseCh and errCh channels and return - if recvErr == io.EOF { + if errors.Is(recvErr, io.EOF) { // skip selection on nil channel errCh = nil close(responseCh) diff --git a/pkg/shuffle/shuffle_test.go b/pkg/shuffle/shuffle_test.go index 159224907..1461fb377 100644 --- a/pkg/shuffle/shuffle_test.go +++ b/pkg/shuffle/shuffle_test.go @@ -120,7 +120,7 @@ func isSameShuffleDistribution(a, b map[int32][]*isb.Message) bool { func buildTestMessagesWithDistinctKeys(size int64) []*isb.Message { // build test messages - messages := testutils.BuildTestWriteMessages(size, time.Now()) + messages := testutils.BuildTestWriteMessages(size, time.Now(), nil) // set keys for test messages var res []*isb.Message for index := 0; index < len(messages); index++ { diff --git a/pkg/sinks/blackhole/blackhole_test.go b/pkg/sinks/blackhole/blackhole_test.go index 637eefbcd..60e9736cd 100644 --- a/pkg/sinks/blackhole/blackhole_test.go +++ b/pkg/sinks/blackhole/blackhole_test.go @@ -36,7 +36,7 @@ func TestBlackhole_Start(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(int64(20), startTime) + writeMessages := testutils.BuildTestWriteMessages(int64(20), startTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ AbstractVertex: dfv1.AbstractVertex{ diff --git a/pkg/sinks/forward/forward_test.go b/pkg/sinks/forward/forward_test.go index 7a7970936..31b9cfd95 100644 --- a/pkg/sinks/forward/forward_test.go +++ b/pkg/sinks/forward/forward_test.go @@ -155,7 +155,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) _, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) fetchWatermark := &testForwardFetcher{} @@ -351,7 +351,7 @@ func TestWriteToBuffer(t *testing.T) { // try to write to buffer after it is full. var messageToStep []isb.Message - writeMessages := testutils.BuildTestWriteMessages(4*value.batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*value.batchSize, testStartTime, nil) messageToStep = append(messageToStep, writeMessages[0:value.batchSize+1]...) _, err = f.writeToBuffer(ctx, buffer, messageToStep) diff --git a/pkg/sinks/forward/shutdown_test.go b/pkg/sinks/forward/shutdown_test.go index 9485b3af4..e2a82b5f7 100644 --- a/pkg/sinks/forward/shutdown_test.go +++ b/pkg/sinks/forward/shutdown_test.go @@ -57,7 +57,7 @@ func TestShutDown(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -97,7 +97,7 @@ func TestShutDown(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", diff --git a/pkg/sinks/logger/log_test.go b/pkg/sinks/logger/log_test.go index 38dd6427a..1ca5e8a64 100644 --- a/pkg/sinks/logger/log_test.go +++ b/pkg/sinks/logger/log_test.go @@ -42,7 +42,7 @@ func TestToLog_Start(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(int64(20), startTime) + writeMessages := testutils.BuildTestWriteMessages(int64(20), startTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ AbstractVertex: dfv1.AbstractVertex{ @@ -121,7 +121,7 @@ func TestToLog_Forward(t *testing.T) { "sinks.logger1": {to1}, } - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) idleManager1, _ := wmb.NewIdleManager(1, 1) diff --git a/pkg/sources/forward/data_forward_test.go b/pkg/sources/forward/data_forward_test.go index 8d01f3916..bed3be734 100644 --- a/pkg/sources/forward/data_forward_test.go +++ b/pkg/sources/forward/data_forward_test.go @@ -157,7 +157,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) fetchWatermark, _ := generic.BuildNoOpSourceWatermarkProgressorsFromBufferMap(toSteps) noOpStores := buildNoOpToVertexStores(toSteps) @@ -246,7 +246,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -397,7 +397,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -562,7 +562,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -711,7 +711,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) fetchWatermark, _ := generic.BuildNoOpSourceWatermarkProgressorsFromBufferMap(toSteps) toVertexStores := buildNoOpToVertexStores(toSteps) @@ -745,7 +745,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -789,7 +789,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -913,7 +913,7 @@ func TestDataForwardSinglePartition(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) fetchWatermark := &testForwardFetcher{} toVertexStores := buildNoOpToVertexStores(toSteps) @@ -968,7 +968,7 @@ func TestDataForwardMultiplePartition(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) fetchWatermark := &testForwardFetcher{} toVertexStores := buildNoOpToVertexStores(toSteps) @@ -1104,7 +1104,7 @@ func TestWriteToBuffer(t *testing.T) { // try to write to buffer after it is full. var messageToStep = make(map[string][][]isb.Message) messageToStep["to1"] = make([][]isb.Message, 1) - writeMessages := testutils.BuildTestWriteMessages(4*value.batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*value.batchSize, testStartTime, nil) messageToStep["to1"][0] = append(messageToStep["to1"][0], writeMessages[0:value.batchSize+1]...) _, err = f.writeToBuffers(ctx, messageToStep) diff --git a/pkg/sources/forward/shutdown_test.go b/pkg/sources/forward/shutdown_test.go index 8e61f345c..160d30502 100644 --- a/pkg/sources/forward/shutdown_test.go +++ b/pkg/sources/forward/shutdown_test.go @@ -73,7 +73,7 @@ func TestInterStepDataForward(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -112,7 +112,7 @@ func TestInterStepDataForward(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", diff --git a/pkg/sources/transformer/grpc_transformer_test.go b/pkg/sources/transformer/grpc_transformer_test.go index 570708273..66259e246 100644 --- a/pkg/sources/transformer/grpc_transformer_test.go +++ b/pkg/sources/transformer/grpc_transformer_test.go @@ -436,7 +436,7 @@ func TestGRPCBasedTransformer_ApplyWithMockClient_ChangePayload(t *testing.T) { u := NewMockGRPCBasedTransformer(mockClient) var count = int64(10) - readMessages := testutils.BuildTestReadMessages(count, time.Unix(1661169600, 0)) + readMessages := testutils.BuildTestReadMessages(count, time.Unix(1661169600, 0), nil) var results = make([][]byte, len(readMessages)) var resultKeys = make([][]string, len(readMessages)) @@ -498,7 +498,7 @@ func TestGRPCBasedTransformer_ApplyWithMockClient_ChangeEventTime(t *testing.T) u := NewMockGRPCBasedTransformer(mockClient) var count = int64(2) - readMessages := testutils.BuildTestReadMessages(count, time.Unix(1661169600, 0)) + readMessages := testutils.BuildTestReadMessages(count, time.Unix(1661169600, 0), nil) for _, readMessage := range readMessages { apply, err := u.ApplyTransform(ctx, &readMessage) assert.NoError(t, err) diff --git a/pkg/udf/forward/forward_test.go b/pkg/udf/forward/forward_test.go index 0adf83dfa..cddef2cf6 100644 --- a/pkg/udf/forward/forward_test.go +++ b/pkg/udf/forward/forward_test.go @@ -139,7 +139,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) @@ -216,7 +216,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -382,7 +382,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -560,7 +560,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -722,7 +722,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) idleManager, _ := wmb.NewIdleManager(1, len(toSteps)) @@ -754,7 +754,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -796,7 +796,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -930,7 +930,7 @@ func TestNewInterStepDataForwardIdleWatermark(t *testing.T) { } ctrlMessage := []isb.Message{{Header: isb.Header{Kind: isb.WMB}}} - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) fetchWatermark := &testWMBFetcher{WMBTestSameHeadWMB: true} toVertexWmStores := buildWatermarkStores(toSteps) @@ -1102,7 +1102,7 @@ func TestNewInterStepDataForwardIdleWatermark_Reset(t *testing.T) { Replica: 0, } - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) fetchWatermark := &testWMBFetcher{WMBTestSameHeadWMB: true} toVertexWmStores := buildWatermarkStores(toSteps) @@ -1350,7 +1350,7 @@ func TestInterStepDataForwardSinglePartition(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) fetchWatermark := &testForwardFetcher{} _, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) @@ -1402,7 +1402,7 @@ func TestInterStepDataForwardMultiplePartition(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) fetchWatermark := &testForwardFetcher{} _, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) @@ -1533,7 +1533,7 @@ func TestWriteToBuffer(t *testing.T) { // try to write to buffer after it is full. var messageToStep = make(map[string][][]isb.Message) messageToStep["to1"] = make([][]isb.Message, 1) - writeMessages := testutils.BuildTestWriteMessages(4*value.batchSize, testStartTime) + writeMessages := testutils.BuildTestWriteMessages(4*value.batchSize, testStartTime, nil) messageToStep["to1"][0] = append(messageToStep["to1"][0], writeMessages[0:value.batchSize+1]...) _, err = f.writeToBuffers(ctx, messageToStep) diff --git a/pkg/udf/forward/shutdown_test.go b/pkg/udf/forward/shutdown_test.go index 3190505e8..1b9d7e460 100644 --- a/pkg/udf/forward/shutdown_test.go +++ b/pkg/udf/forward/shutdown_test.go @@ -76,7 +76,7 @@ func TestInterStepDataForward(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -116,7 +116,7 @@ func TestInterStepDataForward(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil) vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", diff --git a/pkg/udf/reduce_udf.go b/pkg/udf/reduce_udf.go index dad5dc24e..ef2a2d854 100644 --- a/pkg/udf/reduce_udf.go +++ b/pkg/udf/reduce_udf.go @@ -25,6 +25,9 @@ import ( "github.com/numaproj/numaflow-go/pkg/info" "go.uber.org/zap" + alignedfs "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/aligned/fs" + noopwal "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/noop" + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/forwarder" "github.com/numaproj/numaflow/pkg/isb" @@ -32,9 +35,8 @@ import ( "github.com/numaproj/numaflow/pkg/reduce" "github.com/numaproj/numaflow/pkg/reduce/applier" "github.com/numaproj/numaflow/pkg/reduce/pbq" - pbqstore "github.com/numaproj/numaflow/pkg/reduce/pbq/store" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store/aligned/fs" - noopstore "github.com/numaproj/numaflow/pkg/reduce/pbq/store/aligned/noop" + "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/unaligned" + unalignedfs "github.com/numaproj/numaflow/pkg/reduce/pbq/wal/unaligned/fs" "github.com/numaproj/numaflow/pkg/reduce/pnf" "github.com/numaproj/numaflow/pkg/sdkclient" "github.com/numaproj/numaflow/pkg/sdkclient/reducer" @@ -199,7 +201,7 @@ func (u *ReduceUDFProcessor) Start(ctx context.Context) error { } // created watermark related components only if watermark is enabled - // otherwise no op will used + // otherwise no pnFManager will used if !u.VertexInstance.Vertex.Spec.Watermark.Disabled { // create from vertex watermark stores fromVertexWmStores, err = jetstream.BuildFromVertexWatermarkStores(ctx, u.VertexInstance, natsClientPool.NextAvailableClient()) @@ -294,15 +296,19 @@ func (u *ReduceUDFProcessor) Start(ctx context.Context) error { defer func() { _ = shutdown(context.Background()) }() } - // create store manager - var storeManager pbqstore.Manager - if u.VertexInstance.Vertex.Spec.UDF.GroupBy.Storage.NoStore != nil { - storeManager = noopstore.NewNoopStores() - } else { - storeManager = fs.NewFSManager(u.VertexInstance, fs.WithStorePath(dfv1.DefaultStorePath), fs.WithMaxBufferSize(dfv1.DefaultStoreMaxBufferSize), fs.WithSyncDuration(dfv1.DefaultStoreSyncDuration)) + // create noop wal manager + walManager := noopwal.NewNoopStores() + // if the vertex has a persistent volume claim or empty dir, create a file system based wal manager + if u.VertexInstance.Vertex.Spec.UDF.GroupBy.Storage.PersistentVolumeClaim != nil || + u.VertexInstance.Vertex.Spec.UDF.GroupBy.Storage.EmptyDir != nil { + if windower.Type() == window.Aligned { + walManager = alignedfs.NewFSManager(u.VertexInstance) + } else { + walManager = unalignedfs.NewFSManager(ctx, dfv1.DefaultSegmentWALPath, dfv1.DefaultCompactWALPath, u.VertexInstance) + } } - pbqManager, err := pbq.NewManager(ctx, u.VertexInstance.Vertex.Spec.Name, u.VertexInstance.Vertex.Spec.PipelineName, u.VertexInstance.Replica, storeManager, windower.Type()) + pbqManager, err := pbq.NewManager(ctx, u.VertexInstance.Vertex.Spec.Name, u.VertexInstance.Vertex.Spec.PipelineName, u.VertexInstance.Replica, walManager, windower.Type()) if err != nil { log.Errorw("Failed to create pbq manager", zap.Error(err)) return fmt.Errorf("failed to create pbq manager, %w", err) @@ -318,10 +324,50 @@ func (u *ReduceUDFProcessor) Start(ctx context.Context) error { opts = append(opts, reduce.WithAllowedLateness(allowedLateness.Duration)) } - op := pnf.NewPnFManager(ctx, u.VertexInstance, udfApplier, writers, pbqManager, conditionalForwarder, publishWatermark, idleManager, windower) + var pnfOption []pnf.Option + // create and start the compactor if the window type is unaligned + // the compactor will delete the persisted messages which belongs to the materialized window + // create a gc events tracker which tracks the gc events, will be used by the pnf + // to track the gc events and the compactor will delete the persisted messages based on the gc events + if windowType.Session != nil { + gcEventsTracker, err := unalignedfs.NewGCEventsWAL(ctx) + if err != nil { + return fmt.Errorf("failed to create gc events tracker, %w", err) + } + + // close the gc events tracker + defer func() { + err = gcEventsTracker.Close() + if err != nil { + log.Errorw("failed to close gc events tracker", zap.Error(err)) + } + log.Info("GC Events WAL Closed") + }() + + pnfOption = append(pnfOption, pnf.WithGCEventsTracker(gcEventsTracker), pnf.WithWindowType(window.Unaligned)) + + compactor, err := unalignedfs.NewCompactor(ctx, &window.SharedUnalignedPartition, dfv1.DefaultGCEventsWALEventsPath, dfv1.DefaultSegmentWALPath, dfv1.DefaultCompactWALPath) + if err != nil { + return fmt.Errorf("failed to create compactor, %w", err) + } + err = compactor.Start(ctx) + if err != nil { + return fmt.Errorf("failed to start compactor, %w", err) + } + defer func(compactor unaligned.Compactor) { + err = compactor.Stop() + if err != nil { + log.Errorw("failed to stop compactor", zap.Error(err)) + } + log.Info("Compactor Stopped") + }(compactor) + } + + // create the pnf manager + pnFManager := pnf.NewPnFManager(ctx, u.VertexInstance, udfApplier, writers, pbqManager, conditionalForwarder, publishWatermark, idleManager, windower, pnfOption...) // for reduce, we read only from one partition - dataForwarder, err := reduce.NewDataForward(ctx, u.VertexInstance, readers[0], writers, pbqManager, storeManager, conditionalForwarder, fetchWatermark, publishWatermark, windower, idleManager, op, opts...) + dataForwarder, err := reduce.NewDataForward(ctx, u.VertexInstance, readers[0], writers, pbqManager, walManager, conditionalForwarder, fetchWatermark, publishWatermark, windower, idleManager, pnFManager, opts...) if err != nil { return fmt.Errorf("failed get a new DataForward, %w", err) } @@ -329,6 +375,7 @@ func (u *ReduceUDFProcessor) Start(ctx context.Context) error { // read the persisted messages before reading the messages from ISB err = dataForwarder.ReplayPersistedMessages(ctx) if err != nil { + log.Errorw("Failed to read and process persisted messages", zap.Error(err)) return fmt.Errorf("failed to read and process persisted messages, %w", err) } @@ -339,10 +386,12 @@ func (u *ReduceUDFProcessor) Start(ctx context.Context) error { defer wg.Done() dataForwarder.Start() log.Info("Forwarder stopped, exiting reduce udf data processor...") + + // after exiting from pbq write loop, we need to gracefully shut down the pnf manager + pnFManager.Shutdown() }() <-ctx.Done() - log.Info("SIGTERM, exiting...") wg.Wait() diff --git a/pkg/watermark/publish/publisher.go b/pkg/watermark/publish/publisher.go index 62ce9194c..cc558449a 100644 --- a/pkg/watermark/publish/publisher.go +++ b/pkg/watermark/publish/publisher.go @@ -179,7 +179,7 @@ func (p *publish) validateWatermark(wm wmb.Watermark, toVertexPartitionIdx int32 p.log.Debugw("New watermark is updated for the head watermark", zap.Int32("toVertexPartitionIdx", toVertexPartitionIdx), zap.Int64("head", headWM.UnixMilli()), zap.Int64("new", wm.UnixMilli())) p.SetHeadWM(wm, toVertexPartitionIdx) } else if wm.BeforeWatermark(headWM) { - p.log.Infow("Skip publishing the new watermark because it's older than the current watermark", zap.Int32("toVertexPartitionIdx", toVertexPartitionIdx), zap.String("entity", p.entity.GetName()), zap.Int64("head", headWM.UnixMilli()), zap.Int64("new", wm.UnixMilli())) + p.log.Debugw("Skip publishing the new watermark because it's older than the current watermark", zap.Int32("toVertexPartitionIdx", toVertexPartitionIdx), zap.String("entity", p.entity.GetName()), zap.Int64("head", headWM.UnixMilli()), zap.Int64("new", wm.UnixMilli())) return wmb.Watermark{}, true } else { p.log.Debugw("Skip publishing the new watermark because it's the same as the current watermark", zap.Int32("toVertexPartitionIdx", toVertexPartitionIdx), zap.String("entity", p.entity.GetName()), zap.Int64("head", headWM.UnixMilli()), zap.Int64("new", wm.UnixMilli())) diff --git a/pkg/watermark/wmb/wmb_test.go b/pkg/watermark/wmb/wmb_test.go index 1892c5472..a92fecfd0 100644 --- a/pkg/watermark/wmb/wmb_test.go +++ b/pkg/watermark/wmb/wmb_test.go @@ -18,11 +18,9 @@ package wmb import ( "bytes" - "encoding/base64" "encoding/binary" "reflect" "testing" - "time" ) func TestDecodeToWMB(t *testing.T) { @@ -209,24 +207,3 @@ func TestWMB_EncodeToBytes(t *testing.T) { }) } } - -func TestDecodeToWMB2(t *testing.T) { - s := []string{"ABdAroOUX6gX2GLZ34wBAAAAAAAA", "AJ3Im8aYX6gXYCf734wBAAAAAAAA", "ACIrAAKXX6gXoI/a34wBAAAAAAAA"} - - for _, v := range s { - - b, err := base64.StdEncoding.DecodeString(v) - if err != nil { - t.Errorf("DecodeToWMB() error = %v", err) - return - } - - got, err := DecodeToWMB(b) - if err != nil { - t.Errorf("DecodeToWMB() error = %v", err) - return - } - - println(int32(time.Since(time.UnixMilli(got.Watermark)).Minutes())) - } -} diff --git a/pkg/window/strategy/session/session.go b/pkg/window/strategy/session/session.go index 6009fee67..6556b1f2c 100644 --- a/pkg/window/strategy/session/session.go +++ b/pkg/window/strategy/session/session.go @@ -122,6 +122,8 @@ func cloneWindow(win window.TimedWindow) *sessionWindow { endTime: win.EndTime(), slot: win.Slot(), keys: win.Keys(), + partition: win.Partition(), + id: win.ID(), } } diff --git a/test/fixtures/expect.go b/test/fixtures/expect.go index 4d433d7d5..cf5d87096 100644 --- a/test/fixtures/expect.go +++ b/test/fixtures/expect.go @@ -52,7 +52,7 @@ func (t *Expect) SinkContains(sinkName string, targetStr string, opts ...SinkChe return t } -func (t *Expect) SinkNotContains(sinkName string, targetStr string) *Expect { +func (t *Expect) SinkNotContains(sinkName string, targetStr string, opts ...SinkCheckOption) *Expect { t.t.Helper() ctx := context.Background() notContains := RedisNotContains(ctx, t.pipeline.Name, sinkName, targetStr) diff --git a/test/reduce-e2e/reduce_test.go b/test/reduce-one-e2e/reduce_one_test.go similarity index 88% rename from test/reduce-e2e/reduce_test.go rename to test/reduce-one-e2e/reduce_one_test.go index 840cedf57..238254f33 100644 --- a/test/reduce-e2e/reduce_test.go +++ b/test/reduce-one-e2e/reduce_one_test.go @@ -1,3 +1,5 @@ +//go:build test + /* Copyright 2022 The Numaproj Authors. Licensed under the Apache License, Version 2.0 (the "License"); @@ -13,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package reduce_e2e +package reduce_one_e2e import ( "context" @@ -248,49 +250,6 @@ func (r *ReduceSuite) TestComplexSlidingWindowPipeline() { done <- struct{}{} } -func (r *ReduceSuite) TestSimpleSessionPipeline() { - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) - defer cancel() - w := r.Given().Pipeline("@testdata/simple-session-reduce-pipeline.yaml"). - When(). - CreatePipelineAndWait() - defer w.DeletePipelineAndWait() - pipelineName := "simple-session-sum" - - // wait for all the pods to come up - w.Expect().VertexPodsRunning() - - count := 0 - done := make(chan struct{}) - go func() { - // publish messages to source vertex, with event time starting from 60000 - startTime := 60000 - for i := 0; true; i++ { - select { - case <-ctx.Done(): - return - case <-done: - return - default: - if count == 50 { - startTime = startTime + (50 * 1000) - count = 0 - } else { - startTime = startTime + 1000 - } - eventTime := strconv.Itoa(startTime) - w.SendMessageTo(pipelineName, "in", NewHttpPostRequest().WithBody([]byte("1")).WithHeader("X-Numaflow-Event-Time", eventTime)) - count += 1 - } - } - }() - - w.Expect().SinkContains("sink", "50") - done <- struct{}{} -} - -// FIXME(session): add test for keyed session window - func TestReduceSuite(t *testing.T) { suite.Run(t, new(ReduceSuite)) } diff --git a/test/reduce-e2e/testdata/complex-reduce-pipeline.yaml b/test/reduce-one-e2e/testdata/complex-reduce-pipeline.yaml similarity index 100% rename from test/reduce-e2e/testdata/complex-reduce-pipeline.yaml rename to test/reduce-one-e2e/testdata/complex-reduce-pipeline.yaml diff --git a/test/reduce-e2e/testdata/complex-sliding-window-pipeline.yaml b/test/reduce-one-e2e/testdata/complex-sliding-window-pipeline.yaml similarity index 100% rename from test/reduce-e2e/testdata/complex-sliding-window-pipeline.yaml rename to test/reduce-one-e2e/testdata/complex-sliding-window-pipeline.yaml diff --git a/test/reduce-e2e/testdata/simple-keyed-reduce-pipeline.yaml b/test/reduce-one-e2e/testdata/simple-keyed-reduce-pipeline.yaml similarity index 100% rename from test/reduce-e2e/testdata/simple-keyed-reduce-pipeline.yaml rename to test/reduce-one-e2e/testdata/simple-keyed-reduce-pipeline.yaml diff --git a/test/reduce-e2e/testdata/simple-non-keyed-reduce-pipeline.yaml b/test/reduce-one-e2e/testdata/simple-non-keyed-reduce-pipeline.yaml similarity index 100% rename from test/reduce-e2e/testdata/simple-non-keyed-reduce-pipeline.yaml rename to test/reduce-one-e2e/testdata/simple-non-keyed-reduce-pipeline.yaml diff --git a/test/reduce-e2e/testdata/simple-reduce-pipeline-wal.yaml b/test/reduce-one-e2e/testdata/simple-reduce-pipeline-wal.yaml similarity index 100% rename from test/reduce-e2e/testdata/simple-reduce-pipeline-wal.yaml rename to test/reduce-one-e2e/testdata/simple-reduce-pipeline-wal.yaml diff --git a/test/reduce-sdk-e2e/reduce_sdk_test.go b/test/reduce-sdk-e2e/reduce_sdk_test.go deleted file mode 100644 index a22b40a66..000000000 --- a/test/reduce-sdk-e2e/reduce_sdk_test.go +++ /dev/null @@ -1,86 +0,0 @@ -//go:build test - -/* -Copyright 2022 The Numaproj Authors. -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 reduce_sdk_e2e - -import ( - "context" - "fmt" - "strconv" - "testing" - "time" - - "github.com/stretchr/testify/suite" - - . "github.com/numaproj/numaflow/test/fixtures" -) - -type ReduceSDKSuite struct { - E2ESuite -} - -func (s *ReduceSDKSuite) TestReduceStreamGo() { - s.testReduceStream("go") -} - -func (s *ReduceSDKSuite) TestReduceStreamJava() { - s.testReduceStream("java") -} - -func (s *ReduceSDKSuite) testReduceStream(lang string) { - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) - defer cancel() - w := s.Given().Pipeline(fmt.Sprintf("@testdata/reduce-stream/reduce-stream-%s.yaml", lang)). - When(). - CreatePipelineAndWait() - defer w.DeletePipelineAndWait() - pipelineName := fmt.Sprintf("reduce-stream-%s", lang) - - // wait for all the pods to come up - w.Expect().VertexPodsRunning() - - done := make(chan struct{}) - go func() { - // publish messages to source vertex, with event time starting from 60000 - startTime := 60000 - for i := 0; true; i++ { - select { - case <-ctx.Done(): - return - case <-done: - return - default: - eventTime := strconv.Itoa(startTime + i*1000) - w.SendMessageTo(pipelineName, "in", NewHttpPostRequest().WithBody([]byte("3")).WithHeader("X-Numaflow-Event-Time", eventTime)) - } - } - }() - - // The reduce stream application summarizes the input messages and returns the sum when the sum is greater than 100. - // Since we are sending 3s, the first returned message should be 102. - // There should be no other values. - w.Expect().SinkContains("sink", "102") - w.Expect().SinkNotContains("sink", "99") - w.Expect().SinkNotContains("sink", "105") - done <- struct{}{} -} - -// FIXME(session): add test for keyed session window - -func TestSessionSuite(t *testing.T) { - suite.Run(t, new(ReduceSDKSuite)) -} diff --git a/test/reduce-two-e2e/reduce_two_test.go b/test/reduce-two-e2e/reduce_two_test.go new file mode 100644 index 000000000..91351604a --- /dev/null +++ b/test/reduce-two-e2e/reduce_two_test.go @@ -0,0 +1,227 @@ +//go:build test + +/* +Copyright 2022 The Numaproj Authors. +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 reduce_two_e2e + +import ( + "context" + "fmt" + "strconv" + "testing" + "time" + + "github.com/stretchr/testify/suite" + + . "github.com/numaproj/numaflow/test/fixtures" +) + +type ReduceSuite struct { + E2ESuite +} + +func (r *ReduceSuite) TestReduceStreamGo() { + r.testReduceStream("go") +} + +func (r *ReduceSuite) TestReduceStreamJava() { + r.testReduceStream("java") +} + +func (r *ReduceSuite) testReduceStream(lang string) { + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + defer cancel() + w := r.Given().Pipeline(fmt.Sprintf("@testdata/reduce-stream/reduce-stream-%s.yaml", lang)). + When(). + CreatePipelineAndWait() + defer w.DeletePipelineAndWait() + pipelineName := fmt.Sprintf("reduce-stream-%s", lang) + + // wait for all the pods to come up + w.Expect().VertexPodsRunning() + + done := make(chan struct{}) + go func() { + // publish messages to source vertex, with event time starting from 60000 + startTime := 60000 + for i := 0; true; i++ { + select { + case <-ctx.Done(): + return + case <-done: + return + default: + eventTime := strconv.Itoa(startTime + i*1000) + w.SendMessageTo(pipelineName, "in", NewHttpPostRequest().WithBody([]byte("3")).WithHeader("X-Numaflow-Event-Time", eventTime)) + } + } + }() + + // The reduce stream application summarizes the input messages and returns the sum when the sum is greater than 100. + // Since we are sending 3s, the first returned message should be 102. + // There should be no other values. + w.Expect().SinkContains("sink", "102") + w.Expect().SinkNotContains("sink", "99") + w.Expect().SinkNotContains("sink", "105") + done <- struct{}{} +} + +func (r *ReduceSuite) TestSimpleSessionPipeline() { + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + defer cancel() + w := r.Given().Pipeline("@testdata/session-reduce/simple-session-sum-pipeline.yaml"). + When(). + CreatePipelineAndWait() + defer w.DeletePipelineAndWait() + pipelineName := "simple-session-sum" + + // wait for all the pods to come up + w.Expect().VertexPodsRunning() + + count := 0 + done := make(chan struct{}) + go func() { + // publish messages to source vertex, with event time starting from 60000 + startTime := 60000 + for i := 0; true; i++ { + select { + case <-ctx.Done(): + return + case <-done: + return + default: + if count == 10 { + startTime = startTime + (10 * 1000) + count = 0 + } else { + startTime = startTime + 1000 + } + eventTime := strconv.Itoa(startTime) + w.SendMessageTo(pipelineName, "in", NewHttpPostRequest().WithBody([]byte("100")).WithHeader("X-Numaflow-Event-Time", eventTime)) + count += 1 + } + } + }() + + w.Expect().SinkContains("sink", "1000") + done <- struct{}{} +} + +func (r *ReduceSuite) TestSimpleSessionKeyedPipeline() { + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + defer cancel() + w := r.Given().Pipeline("@testdata/session-reduce/simple-session-keyed-counter-pipeline.yaml"). + When(). + CreatePipelineAndWait() + defer w.DeletePipelineAndWait() + pipelineName := "simple-session-counter" + + // wait for all the pods to come up + w.Expect().VertexPodsRunning() + + count := 0 + done := make(chan struct{}) + go func() { + // publish messages to source vertex, with event time starting from 60000 + startTime := 60000 + for i := 0; true; i++ { + select { + case <-ctx.Done(): + return + case <-done: + return + default: + if count == 10 { + startTime = startTime + (10 * 1000) + count = 0 + } else { + startTime = startTime + 1000 + } + eventTime := strconv.Itoa(startTime) + w.SendMessageTo(pipelineName, "in", NewHttpPostRequest().WithBody([]byte("1")).WithHeader("X-Numaflow-Event-Time", eventTime)) + w.SendMessageTo(pipelineName, "in", NewHttpPostRequest().WithBody([]byte("2")).WithHeader("X-Numaflow-Event-Time", eventTime)) + count += 2 + } + } + }() + + w.Expect().SinkContains("sink", "5") + w.Expect().SinkNotContains("sink", "4", WithTimeout(20*time.Second)) + w.Expect().SinkNotContains("sink", "3", WithTimeout(20*time.Second)) + w.Expect().SinkNotContains("sink", "2", WithTimeout(20*time.Second)) + w.Expect().SinkNotContains("sink", "1", WithTimeout(20*time.Second)) + done <- struct{}{} +} + +func (r *ReduceSuite) TestSimpleSessionPipelineFailOverUsingWAL() { + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + defer cancel() + w := r.Given().Pipeline("@testdata/session-reduce/simple-session-keyed-counter-pipeline.yaml"). + When(). + CreatePipelineAndWait() + defer w.DeletePipelineAndWait() + pipelineName := "simple-session-counter" + + // wait for all the pods to come up + w.Expect().VertexPodsRunning() + + args := "kubectl delete po -n numaflow-system -l " + + "numaflow.numaproj.io/pipeline-name=simple-session-counter,numaflow.numaproj.io/vertex-name=compute-count" + + // Kill the reducer pods before processing to trigger failover. + w.Exec("/bin/sh", []string{"-c", args}, CheckPodKillSucceeded) + done := make(chan struct{}) + count := 0 + go func() { + startTime := 0 + for i := 1; true; i++ { + select { + case <-ctx.Done(): + return + case <-done: + return + default: + if count == 10 { + startTime = startTime + (10 * 1000) + count = 0 + } else { + startTime = startTime + 1000 + } + eventTime := strconv.Itoa(startTime) + if i == 5 { + // Kill the reducer pods during processing to trigger failover. + w.Expect().VertexPodsRunning() + w.Exec("/bin/sh", []string{"-c", args}, CheckPodKillSucceeded) + } + w.SendMessageTo(pipelineName, "in", NewHttpPostRequest().WithBody([]byte("1")).WithHeader("X-Numaflow-Event-Time", eventTime)). + SendMessageTo(pipelineName, "in", NewHttpPostRequest().WithBody([]byte("2")).WithHeader("X-Numaflow-Event-Time", eventTime)) + count += 2 + } + } + }() + + w.Expect(). + SinkContains("sink", "5"). + SinkNotContains("sink", "4", WithTimeout(20*time.Second)). + SinkNotContains("sink", "3", WithTimeout(20*time.Second)). + SinkNotContains("sink", "2", WithTimeout(20*time.Second)). + SinkNotContains("sink", "1", WithTimeout(20*time.Second)) + done <- struct{}{} +} + +func TestSessionSuite(t *testing.T) { + suite.Run(t, new(ReduceSuite)) +} diff --git a/test/reduce-sdk-e2e/testdata/reduce-stream/reduce-stream-go.yaml b/test/reduce-two-e2e/testdata/reduce-stream/reduce-stream-go.yaml similarity index 100% rename from test/reduce-sdk-e2e/testdata/reduce-stream/reduce-stream-go.yaml rename to test/reduce-two-e2e/testdata/reduce-stream/reduce-stream-go.yaml diff --git a/test/reduce-sdk-e2e/testdata/reduce-stream/reduce-stream-java.yaml b/test/reduce-two-e2e/testdata/reduce-stream/reduce-stream-java.yaml similarity index 100% rename from test/reduce-sdk-e2e/testdata/reduce-stream/reduce-stream-java.yaml rename to test/reduce-two-e2e/testdata/reduce-stream/reduce-stream-java.yaml diff --git a/test/reduce-two-e2e/testdata/session-reduce/simple-session-keyed-counter-pipeline.yaml b/test/reduce-two-e2e/testdata/session-reduce/simple-session-keyed-counter-pipeline.yaml new file mode 100644 index 000000000..6966bce86 --- /dev/null +++ b/test/reduce-two-e2e/testdata/session-reduce/simple-session-keyed-counter-pipeline.yaml @@ -0,0 +1,47 @@ +apiVersion: numaflow.numaproj.io/v1alpha1 +kind: Pipeline +metadata: + name: simple-session-counter +spec: + vertices: + - name: in + scale: + min: 1 + source: + http: {} + - name: even-odd + scale: + min: 1 + udf: + container: + image: quay.io/numaio/numaflow-go/map-even-odd:v0.6.1 + - name: compute-count + partitions: 1 + udf: + container: + # see https://github.com/numaproj/numaflow-go/tree/main/pkg/sessionreducer/examples/counter + image: quay.io/numaio/numaflow-go/session-counter:v0.6.1 + groupBy: + window: + session: + timeout: 10s + keyed: true + storage: + persistentVolumeClaim: + volumeSize: 2Gi + accessMode: ReadWriteOnce + - name: sink + scale: + min: 1 + sink: + udsink: + container: + # A redis sink for e2e testing, see https://github.com/numaproj/numaflow-sinks/tree/main/redis-e2e-test-sink + image: quay.io/numaio/numaflow-sink/redis-e2e-test-sink:latest + edges: + - from: in + to: even-odd + - from: even-odd + to: compute-count + - from: compute-count + to: sink diff --git a/test/reduce-e2e/testdata/simple-session-reduce-pipeline.yaml b/test/reduce-two-e2e/testdata/session-reduce/simple-session-sum-pipeline.yaml similarity index 93% rename from test/reduce-e2e/testdata/simple-session-reduce-pipeline.yaml rename to test/reduce-two-e2e/testdata/session-reduce/simple-session-sum-pipeline.yaml index 62d1b7782..2b493e3e5 100644 --- a/test/reduce-e2e/testdata/simple-session-reduce-pipeline.yaml +++ b/test/reduce-two-e2e/testdata/session-reduce/simple-session-sum-pipeline.yaml @@ -14,7 +14,7 @@ spec: udf: container: # see https://github.com/numaproj/numaflow-go/tree/main/pkg/sessionreducer/examples/counter - image: quay.io/numaio/numaflow-go/session-counter:v0.6.1 + image: quay.io/numaio/numaflow-go/session-sum:v0.6.1 imagePullPolicy: Always groupBy: window: