From d517d30dea7e7efdcac9a2b5b32077b1667a8067 Mon Sep 17 00:00:00 2001 From: Neil Twigg Date: Mon, 2 Dec 2024 11:39:53 +0000 Subject: [PATCH 1/2] Initial support for per-message TTLs Signed-off-by: Neil Twigg --- server/errors.json | 10 + server/filestore.go | 252 ++++++++- server/filestore_test.go | 783 +++++++++++++++++---------- server/jetstream_cluster.go | 2 +- server/jetstream_errors_generated.go | 14 + server/jetstream_helpers_test.go | 33 ++ server/jetstream_test.go | 204 +++++++ server/memstore.go | 10 +- server/memstore_test.go | 90 +-- server/norace_test.go | 44 +- server/raft.go | 4 +- server/store.go | 4 +- server/store_test.go | 8 +- server/stream.go | 53 +- server/thw/thw.go | 3 + 15 files changed, 1107 insertions(+), 407 deletions(-) diff --git a/server/errors.json b/server/errors.json index 9bb8cb18c66..d818d403306 100644 --- a/server/errors.json +++ b/server/errors.json @@ -1628,5 +1628,15 @@ "help": "", "url": "", "deprecates": "" + }, + { + "constant": "JSMessageTTLInvalidErr", + "code": 400, + "error_code": 10165, + "description": "invalid per-message TTL", + "comment": "", + "help": "", + "url": "", + "deprecates": "" } ] diff --git a/server/filestore.go b/server/filestore.go index 7d4b2637e3a..f9fd375ce59 100644 --- a/server/filestore.go +++ b/server/filestore.go @@ -45,6 +45,7 @@ import ( "github.com/minio/highwayhash" "github.com/nats-io/nats-server/v2/server/avl" "github.com/nats-io/nats-server/v2/server/stree" + "github.com/nats-io/nats-server/v2/server/thw" "golang.org/x/crypto/chacha20" "golang.org/x/crypto/chacha20poly1305" ) @@ -68,6 +69,8 @@ type FileStoreConfig struct { Cipher StoreCipher // Compression is the algorithm to use when compressing. Compression StoreCompression + // EnforceTTLs decides whether or not to enforce per-message TTLs. + EnforceTTLs bool // Internal reference to our server. srv *Server @@ -198,6 +201,8 @@ type fileStore struct { fip bool receivedAny bool firstMoved bool + ttls *thw.HashWheel + ttlseq uint64 // How up-to-date is the `ttls` THW? } // Represents a message store block and its data. @@ -244,6 +249,7 @@ type msgBlock struct { syncAlways bool noCompact bool closed bool + ttls uint64 // How many msgs have TTLs? // Used to mock write failures. mockWriteErr bool @@ -321,6 +327,9 @@ const ( // This is the full snapshotted state for the stream. streamStreamStateFile = "index.db" + // This is the encoded time hash wheel for TTLs. + ttlStreamStateFile = "thw.db" + // AEK key sizes minMetaKeySize = 64 minBlkKeySize = 64 @@ -408,6 +417,11 @@ func newFileStoreWithCreated(fcfg FileStoreConfig, cfg StreamConfig, created tim srv: fcfg.srv, } + // Only create a THW if we're going to allow TTLs. + if fs.fcfg.EnforceTTLs { + fs.ttls = thw.NewHashWheel() + } + // Set flush in place to AsyncFlush which by default is false. fs.fip = !fcfg.AsyncFlush @@ -471,6 +485,13 @@ func newFileStoreWithCreated(fcfg FileStoreConfig, cfg StreamConfig, created tim fs.dirty++ } + // See if we can bring back our TTL timed hash wheel state from disk. + if fcfg.EnforceTTLs { + if err = fs.recoverTTLState(); err != nil && !os.IsNotExist(err) { + fs.warn("Recovering TTL state from index errored: %v", err) + } + } + // Also make sure we get rid of old idx and fss files on return. // Do this in separate go routine vs inline and at end of processing. defer func() { @@ -1380,6 +1401,10 @@ func (mb *msgBlock) rebuildStateLocked() (*LostStreamData, []uint64, error) { rl, slen := le.Uint32(hdr[0:]), int(le.Uint16(hdr[20:])) hasHeaders := rl&hbit != 0 + var ttl int64 + if mb.fs.ttls != nil && len(hdr) > 0 { + ttl, _ = getMessageTTL(hdr) + } // Clear any headers bit that could be set. rl &^= hbit dlen := int(rl) - msgHdrSize @@ -1455,6 +1480,11 @@ func (mb *msgBlock) rebuildStateLocked() (*LostStreamData, []uint64, error) { if !mb.dmap.Exists(seq) { mb.msgs++ mb.bytes += uint64(rl) + if mb.fs.ttls != nil && ttl > 0 { + expires := time.Duration(ts) + (time.Second * time.Duration(ttl)) + mb.fs.ttls.Add(seq, int64(expires)) + mb.ttls++ + } } // Check for any gaps from compaction, meaning no ebit entry. @@ -1593,7 +1623,8 @@ func (fs *fileStore) recoverFullState() (rerr error) { } } - if buf[0] != fullStateMagic || buf[1] != fullStateVersion { + version := buf[1] + if buf[0] != fullStateMagic || version < fullStateMinVersion || version > fullStateVersion { os.Remove(fn) fs.warn("Stream state magic and version mismatch") return errCorruptState @@ -1685,6 +1716,10 @@ func (fs *fileStore) recoverFullState() (rerr error) { fs.blks = make([]*msgBlock, 0, numBlocks) for i := 0; i < int(numBlocks); i++ { index, nbytes, fseq, fts, lseq, lts, numDeleted := uint32(readU64()), readU64(), readU64(), readI64(), readU64(), readI64(), readU64() + var ttls uint64 + if version >= 2 { + ttls = readU64() + } if bi < 0 { os.Remove(fn) return errCorruptState @@ -1694,6 +1729,7 @@ func (fs *fileStore) recoverFullState() (rerr error) { atomic.StoreUint64(&mb.last.seq, lseq) mb.msgs, mb.bytes = lseq-fseq+1, nbytes mb.first.ts, mb.last.ts = fts+baseTime, lts+baseTime + mb.ttls = ttls if numDeleted > 0 { dmap, n, err := avl.Decode(buf[bi:]) if err != nil { @@ -1794,6 +1830,81 @@ func (fs *fileStore) recoverFullState() (rerr error) { return nil } +func (fs *fileStore) recoverTTLState() error { + // See if we have a timed hash wheel for TTLs. + <-dios + fn := filepath.Join(fs.fcfg.StoreDir, msgDir, ttlStreamStateFile) + buf, err := os.ReadFile(fn) + dios <- struct{}{} + + if err != nil && !os.IsNotExist(err) { + return err + } + + fs.ttls = thw.NewHashWheel() + + if err == nil { + fs.ttlseq, err = fs.ttls.Decode(buf) + if err != nil { + fs.warn("Error decoding TTL state: %s", err) + os.Remove(fn) + } + } + + if fs.ttlseq < fs.state.FirstSeq { + fs.ttlseq = fs.state.FirstSeq + } + + defer fs.resetAgeChk(0) + if fs.ttlseq <= fs.state.LastSeq { + fs.warn("TTL state is outdated; attempting to recover using linear scan (seq %d to %d)", fs.ttlseq, fs.state.LastSeq) + var sm StoreMsg + mb := fs.selectMsgBlock(fs.ttlseq) + if mb == nil { + return nil + } + mblseq := atomic.LoadUint64(&mb.last.seq) + for seq := fs.ttlseq; seq <= fs.state.LastSeq; seq++ { + retry: + if mb.ttls == 0 { + // None of the messages in the block have message TTLs so don't + // bother doing anything further with this block, skip to the end. + seq = atomic.LoadUint64(&mb.last.seq) + 1 + } + if seq > mblseq { + // We've reached the end of the loaded block, see if we can continue + // by loading the next one. + mb.tryForceExpireCache() + if mb = fs.selectMsgBlock(seq); mb == nil { + // TODO(nat): Deal with gaps properly. Right now this will be + // probably expensive on CPU. + continue + } + mblseq = atomic.LoadUint64(&mb.last.seq) + // At this point we've loaded another block, so let's go back to the + // beginning and see if we need to skip this one too. + goto retry + } + msg, _, err := mb.fetchMsg(seq, &sm) + if err != nil { + fs.warn("Error loading msg seq %d for recovering TTL: %s", seq, err) + continue + } + if len(msg.hdr) == 0 { + continue + } + if ttl, _ := getMessageTTL(msg.hdr); ttl > 0 { + expires := time.Duration(msg.ts) + (time.Second * time.Duration(ttl)) + fs.ttls.Add(seq, int64(expires)) + if seq > fs.ttlseq { + fs.ttlseq = seq + } + } + } + } + return nil +} + // Grabs last checksum for the named block file. // Takes into account encryption etc. func (mb *msgBlock) lastChecksum() []byte { @@ -3625,7 +3736,7 @@ func (fs *fileStore) genEncryptionKeysForBlock(mb *msgBlock) error { // Stores a raw message with expected sequence number and timestamp. // Lock should be held. -func (fs *fileStore) storeRawMsg(subj string, hdr, msg []byte, seq uint64, ts int64) (err error) { +func (fs *fileStore) storeRawMsg(subj string, hdr, msg []byte, seq uint64, ts, ttl int64) (err error) { if fs.closed { return ErrStoreClosed } @@ -3677,6 +3788,8 @@ func (fs *fileStore) storeRawMsg(subj string, hdr, msg []byte, seq uint64, ts in } // Write msg record. + // Add expiry bit to sequence if needed. This is so that if we need to + // rebuild, we know which messages to look at more quickly. n, err := fs.writeMsgRecord(seq, ts, subj, hdr, msg) if err != nil { return err @@ -3744,18 +3857,31 @@ func (fs *fileStore) storeRawMsg(subj string, hdr, msg []byte, seq uint64, ts in fs.enforceMsgLimit() fs.enforceBytesLimit() + // Per-message TTL. + if fs.ttls != nil && ttl > 0 { + expires := time.Duration(ts) + (time.Second * time.Duration(ttl)) + fs.ttls.Add(seq, int64(expires)) + fs.lmb.ttls++ + if seq > fs.ttlseq { + fs.ttlseq = seq + } + } + // Check if we have and need the age expiration timer running. - if fs.ageChk == nil && fs.cfg.MaxAge != 0 { + switch { + case fs.ageChk == nil && (fs.cfg.MaxAge > 0 || fs.ttls != nil): fs.startAgeChk() + case fs.ageChk != nil && fs.ttls != nil && ttl > 0: + fs.resetAgeChk(0) } return nil } // StoreRawMsg stores a raw message with expected sequence number and timestamp. -func (fs *fileStore) StoreRawMsg(subj string, hdr, msg []byte, seq uint64, ts int64) error { +func (fs *fileStore) StoreRawMsg(subj string, hdr, msg []byte, seq uint64, ts, ttl int64) error { fs.mu.Lock() - err := fs.storeRawMsg(subj, hdr, msg, seq, ts) + err := fs.storeRawMsg(subj, hdr, msg, seq, ts, ttl) cb := fs.scb // Check if first message timestamp requires expiry // sooner than initial replica expiry timer set to MaxAge when initializing. @@ -3775,10 +3901,10 @@ func (fs *fileStore) StoreRawMsg(subj string, hdr, msg []byte, seq uint64, ts in } // Store stores a message. We hold the main filestore lock for any write operation. -func (fs *fileStore) StoreMsg(subj string, hdr, msg []byte) (uint64, int64, error) { +func (fs *fileStore) StoreMsg(subj string, hdr, msg []byte, ttl int64) (uint64, int64, error) { fs.mu.Lock() seq, ts := fs.state.LastSeq+1, time.Now().UnixNano() - err := fs.storeRawMsg(subj, hdr, msg, seq, ts) + err := fs.storeRawMsg(subj, hdr, msg, seq, ts, ttl) cb := fs.scb fs.mu.Unlock() @@ -5107,25 +5233,52 @@ func (mb *msgBlock) expireCacheLocked() { } func (fs *fileStore) startAgeChk() { - if fs.ageChk == nil && fs.cfg.MaxAge != 0 { + if fs.ageChk != nil { + return + } + if fs.cfg.MaxAge != 0 || fs.ttls != nil { fs.ageChk = time.AfterFunc(fs.cfg.MaxAge, fs.expireMsgs) } } // Lock should be held. func (fs *fileStore) resetAgeChk(delta int64) { - if fs.cfg.MaxAge == 0 { + var next int64 = math.MaxInt64 + if fs.ttls != nil { + next = fs.ttls.GetNextExpiration(next) + } + + // If there's no MaxAge and there's nothing waiting to be expired then + // don't bother continuing. The next storeRawMsg() will wake us up if + // needs be. + if fs.cfg.MaxAge <= 0 && next == math.MaxInt64 { + clearTimer(&fs.ageChk) return } + // Check to see if we should be firing sooner than MaxAge for an expiring TTL. fireIn := fs.cfg.MaxAge - if delta > 0 && time.Duration(delta) < fireIn { - if fireIn = time.Duration(delta); fireIn < 250*time.Millisecond { - // Only fire at most once every 250ms. - // Excessive firing can effect ingest performance. - fireIn = time.Second + if next < math.MaxInt64 { + // Looks like there's a next expiration, use it either if there's no + // MaxAge set or if it looks to be sooner than MaxAge is. + if until := time.Until(time.Unix(0, next)); fireIn == 0 || until < fireIn { + fireIn = until + } + } + + // If not then look at the delta provided (usually gap to next age expiry). + if delta > 0 { + if fireIn == 0 || time.Duration(delta) < fireIn { + fireIn = time.Duration(delta) } } + + // Make sure we aren't firing too often either way, otherwise we can + // negatively impact stream ingest performance. + if fireIn < 250*time.Millisecond { + fireIn = 250 * time.Millisecond + } + if fs.ageChk != nil { fs.ageChk.Reset(fireIn) } else { @@ -5152,19 +5305,30 @@ func (fs *fileStore) expireMsgs() { minAge := time.Now().UnixNano() - maxAge fs.mu.RUnlock() - for sm, _ = fs.msgForSeq(0, &smv); sm != nil && sm.ts <= minAge; sm, _ = fs.msgForSeq(0, &smv) { - fs.mu.Lock() - fs.removeMsgViaLimits(sm.seq) - fs.mu.Unlock() - // Recalculate in case we are expiring a bunch. - minAge = time.Now().UnixNano() - maxAge + if maxAge > 0 { + for sm, _ = fs.msgForSeq(0, &smv); sm != nil && sm.ts <= minAge; sm, _ = fs.msgForSeq(0, &smv) { + fs.mu.Lock() + fs.removeMsgViaLimits(sm.seq) + fs.mu.Unlock() + // Recalculate in case we are expiring a bunch. + minAge = time.Now().UnixNano() - maxAge + } } fs.mu.Lock() defer fs.mu.Unlock() + // TODO: Not great that we're holding the lock here, but the timed hash wheel isn't thread-safe. + var nextTTL int64 + if fs.ttls != nil { + fs.ttls.ExpireTasks(func(seq uint64, ts int64) { + fs.removeMsgViaLimits(seq) + }) + nextTTL = fs.ttls.GetNextExpiration(math.MaxInt64) + } + // Onky cancel if no message left, not on potential lookup error that would result in sm == nil. - if fs.state.Msgs == 0 { + if fs.state.Msgs == 0 && nextTTL == math.MaxInt64 { fs.cancelAgeChk() } else { if sm == nil { @@ -5929,9 +6093,11 @@ func (mb *msgBlock) indexCacheBuf(buf []byte) error { } // Mark fss activity. mb.lsts = time.Now().UnixNano() + mb.ttls = 0 lbuf := uint32(len(buf)) - var seq uint64 + var seq, ttls uint64 + var sm StoreMsg // Used for finding TTL headers for index < lbuf { if index+msgHdrSize > lbuf { return errCorruptState @@ -5941,6 +6107,7 @@ func (mb *msgBlock) indexCacheBuf(buf []byte) error { seq = le.Uint64(hdr[4:]) // Clear any headers bit that could be set. + hasHeaders := rl&hbit != 0 rl &^= hbit dlen := int(rl) - msgHdrSize @@ -6002,6 +6169,16 @@ func (mb *msgBlock) indexCacheBuf(buf []byte) error { }) } } + + // Count how many TTLs we think are in this message block. + // TODO(nat): Not terribly optimal... + if hasHeaders { + if fsm, err := mb.msgFromBuf(buf, &sm, nil); err == nil && fsm != nil { + if _, err = getMessageTTL(fsm.hdr); err == nil && len(fsm.hdr) > 0 { + ttls++ + } + } + } } index += rl } @@ -6023,6 +6200,7 @@ func (mb *msgBlock) indexCacheBuf(buf []byte) error { mb.cache.idx = idx mb.cache.fseq = fseq mb.cache.wp += int(lbuf) + mb.ttls = ttls return nil } @@ -8317,9 +8495,13 @@ func (fs *fileStore) cancelSyncTimer() { } } +// The full state file is versioned. +// - 0x1: original binary index.db format +// - 0x2: adds support for TTL count field after num deleted const ( - fullStateMagic = uint8(11) - fullStateVersion = uint8(1) + fullStateMagic = uint8(11) + fullStateMinVersion = uint8(1) // What is the minimum version we know how to parse? + fullStateVersion = uint8(2) // What is the current version written out to index.db? ) // This go routine periodically writes out our full stream state index. @@ -8411,7 +8593,7 @@ func (fs *fileStore) _writeFullState(force bool) error { binary.MaxVarintLen64 + fs.tsl + // NumSubjects + total subject length numSubjects*(binary.MaxVarintLen64*4) + // psi record binary.MaxVarintLen64 + // Num blocks. - len(fs.blks)*((binary.MaxVarintLen64*7)+avgDmapLen) + // msg blocks, avgDmapLen is est for dmaps + len(fs.blks)*((binary.MaxVarintLen64*8)+avgDmapLen) + // msg blocks, avgDmapLen is est for dmaps binary.MaxVarintLen64 + 8 + 8 // last index + record checksum + full state checksum // Do 4k on stack if possible. @@ -8473,6 +8655,7 @@ func (fs *fileStore) _writeFullState(force bool) error { numDeleted := mb.dmap.Size() buf = binary.AppendUvarint(buf, uint64(numDeleted)) + buf = binary.AppendUvarint(buf, mb.ttls) // Field is new in version 2 if numDeleted > 0 { dmap, _ := mb.dmap.Encode(scratch[:0]) dmapTotalLen += len(dmap) @@ -8554,7 +8737,24 @@ func (fs *fileStore) _writeFullState(force bool) error { fs.mu.Unlock() } - return nil + return fs.writeTTLState() +} + +func (fs *fileStore) writeTTLState() error { + if fs.ttls == nil { + return nil + } + + fs.mu.RLock() + fn := filepath.Join(fs.fcfg.StoreDir, msgDir, ttlStreamStateFile) + buf := fs.ttls.Encode(fs.state.LastSeq) + fs.mu.RUnlock() + + <-dios + err := os.WriteFile(fn, buf, defaultFilePerms) + dios <- struct{}{} + + return err } // Stop the current filestore. diff --git a/server/filestore_test.go b/server/filestore_test.go index 34a7862440d..6ca01973597 100644 --- a/server/filestore_test.go +++ b/server/filestore_test.go @@ -84,7 +84,7 @@ func TestFileStoreBasics(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 1; i <= 5; i++ { now := time.Now().UnixNano() - if seq, ts, err := fs.StoreMsg(subj, nil, msg); err != nil { + if seq, ts, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Error storing msg: %v", err) } else if seq != uint64(i) { t.Fatalf("Expected sequence to be %d, got %d", i, seq) @@ -152,7 +152,7 @@ func TestFileStoreMsgHeaders(t *testing.T) { if sz := int(fileStoreMsgSize(subj, hdr, msg)); sz != elen { t.Fatalf("Wrong size for stored msg with header") } - fs.StoreMsg(subj, hdr, msg) + fs.StoreMsg(subj, hdr, msg, 0) var smv StoreMsg sm, err := fs.LoadMsg(1, &smv) if err != nil { @@ -190,7 +190,7 @@ func TestFileStoreBasicWriteMsgsAndRestore(t *testing.T) { toStore := uint64(100) for i := uint64(1); i <= toStore; i++ { msg := []byte(fmt.Sprintf("[%08d] Hello World!", i)) - if seq, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if seq, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Error storing msg: %v", err) } else if seq != uint64(i) { t.Fatalf("Expected sequence to be %d, got %d", i, seq) @@ -210,7 +210,7 @@ func TestFileStoreBasicWriteMsgsAndRestore(t *testing.T) { fs.Stop() // Make sure Store call after does not work. - if _, _, err := fs.StoreMsg(subj, nil, []byte("no work")); err == nil { + if _, _, err := fs.StoreMsg(subj, nil, []byte("no work"), 0); err == nil { t.Fatalf("Expected an error for StoreMsg call after Stop, got none") } @@ -230,7 +230,7 @@ func TestFileStoreBasicWriteMsgsAndRestore(t *testing.T) { // Now write 100 more msgs for i := uint64(101); i <= toStore*2; i++ { msg := []byte(fmt.Sprintf("[%08d] Hello World!", i)) - if seq, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if seq, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Error storing msg: %v", err) } else if seq != uint64(i) { t.Fatalf("Expected sequence to be %d, got %d", i, seq) @@ -274,7 +274,7 @@ func TestFileStoreBasicWriteMsgsAndRestore(t *testing.T) { t.Fatalf("Expected %d bytes, got %d", 0, state.Bytes) } - seq, _, err := fs.StoreMsg(subj, nil, []byte("Hello")) + seq, _, err := fs.StoreMsg(subj, nil, []byte("Hello"), 0) if err != nil { t.Fatalf("Unexpected error: %v", err) } @@ -302,7 +302,7 @@ func TestFileStoreSelectNextFirst(t *testing.T) { numMsgs := 10 subj, msg := "zzz", []byte("Hello World") for i := 0; i < numMsgs; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } if state := fs.State(); state.Msgs != uint64(numMsgs) { t.Fatalf("Expected %d msgs, got %d", numMsgs, state.Msgs) @@ -349,10 +349,10 @@ func TestFileStoreSkipMsg(t *testing.T) { t.Fatalf("Expected first to be %d and last to be %d. got first %d and last %d", numSkips+1, numSkips, state.FirstSeq, state.LastSeq) } - fs.StoreMsg("zzz", nil, []byte("Hello World!")) + fs.StoreMsg("zzz", nil, []byte("Hello World!"), 0) fs.SkipMsg() fs.SkipMsg() - fs.StoreMsg("zzz", nil, []byte("Hello World!")) + fs.StoreMsg("zzz", nil, []byte("Hello World!"), 0) fs.SkipMsg() state = fs.State() @@ -388,7 +388,7 @@ func TestFileStoreSkipMsg(t *testing.T) { } fs.SkipMsg() - nseq, _, err := fs.StoreMsg("AAA", nil, []byte("Skip?")) + nseq, _, err := fs.StoreMsg("AAA", nil, []byte("Skip?"), 0) if err != nil { t.Fatalf("Unexpected error looking up seq 11: %v", err) } @@ -424,7 +424,7 @@ func TestFileStoreWriteExpireWrite(t *testing.T) { toSend := 10 for i := 0; i < toSend; i++ { - fs.StoreMsg("zzz", nil, []byte("Hello World!")) + fs.StoreMsg("zzz", nil, []byte("Hello World!"), 0) } // Wait for write cache portion to go to zero. @@ -436,7 +436,7 @@ func TestFileStoreWriteExpireWrite(t *testing.T) { }) for i := 0; i < toSend; i++ { - fs.StoreMsg("zzz", nil, []byte("Hello World! - 22")) + fs.StoreMsg("zzz", nil, []byte("Hello World! - 22"), 0) } if state := fs.State(); state.Msgs != uint64(toSend*2) { @@ -481,13 +481,13 @@ func TestFileStoreMsgLimit(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 0; i < 10; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state := fs.State() if state.Msgs != 10 { t.Fatalf("Expected %d msgs, got %d", 10, state.Msgs) } - if _, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Error storing msg: %v", err) } state = fs.State() @@ -515,14 +515,14 @@ func TestFileStoreMsgLimitBug(t *testing.T) { defer fs.Stop() subj, msg := "foo", []byte("Hello World") - fs.StoreMsg(subj, nil, msg) - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) + fs.StoreMsg(subj, nil, msg, 0) fs.Stop() fs, err = newFileStoreWithCreated(fcfg, StreamConfig{Name: "zzz", Storage: FileStorage, MaxMsgs: 1}, created, prf(&fcfg), nil) require_NoError(t, err) defer fs.Stop() - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) }) } @@ -539,7 +539,7 @@ func TestFileStoreBytesLimit(t *testing.T) { defer fs.Stop() for i := uint64(0); i < toStore; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state := fs.State() if state.Msgs != toStore { @@ -551,7 +551,7 @@ func TestFileStoreBytesLimit(t *testing.T) { // Now send 10 more and check that bytes limit enforced. for i := 0; i < 10; i++ { - if _, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Error storing msg: %v", err) } } @@ -586,7 +586,7 @@ func TestFileStoreBytesLimitWithDiscardNew(t *testing.T) { defer fs.Stop() for i := 0; i < 10; i++ { - _, _, err := fs.StoreMsg(subj, nil, msg) + _, _, err := fs.StoreMsg(subj, nil, msg, 0) if i < int(toStore) { if err != nil { t.Fatalf("Error storing msg: %v", err) @@ -626,7 +626,7 @@ func TestFileStoreAgeLimit(t *testing.T) { subj, msg := "foo", []byte("Hello World") toStore := 500 for i := 0; i < toStore; i++ { - if _, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Unexpected error: %v", err) } } @@ -652,7 +652,7 @@ func TestFileStoreAgeLimit(t *testing.T) { // Now add some more and make sure that timer will fire again. for i := 0; i < toStore; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state = fs.State() if state.Msgs != uint64(toStore) { @@ -681,7 +681,7 @@ func TestFileStoreTimeStamps(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 0; i < 10; i++ { time.Sleep(5 * time.Millisecond) - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } var smv StoreMsg for seq := uint64(1); seq <= 10; seq++ { @@ -713,7 +713,7 @@ func TestFileStorePurge(t *testing.T) { toStore := uint64(1024) for i := uint64(0); i < toStore; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state := fs.State() if state.Msgs != toStore { @@ -767,7 +767,7 @@ func TestFileStorePurge(t *testing.T) { // Now make sure we clean up any dangling purged messages. for i := uint64(0); i < toStore; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state = fs.State() if state.Msgs != toStore { @@ -821,7 +821,7 @@ func TestFileStoreCompact(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 0; i < 10; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } if state := fs.State(); state.Msgs != 10 { t.Fatalf("Expected 10 msgs, got %d", state.Msgs) @@ -875,7 +875,7 @@ func TestFileStoreCompactLastPlusOne(t *testing.T) { subj, msg := "foo", make([]byte, 10_000) for i := 0; i < 10_000; i++ { - if _, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Unexpected error: %v", err) } } @@ -897,7 +897,7 @@ func TestFileStoreCompactLastPlusOne(t *testing.T) { t.Fatalf("Expected no message but got %d", state.Msgs) } - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) state = fs.State() if state.Msgs != 1 { t.Fatalf("Expected one message but got %d", state.Msgs) @@ -913,7 +913,7 @@ func TestFileStoreCompactMsgCountBug(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 0; i < 10; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } if state := fs.State(); state.Msgs != 10 { t.Fatalf("Expected 10 msgs, got %d", state.Msgs) @@ -955,7 +955,7 @@ func TestFileStoreCompactPerf(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 0; i < 100_000; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } if state := fs.State(); state.Msgs != 100_000 { t.Fatalf("Expected 1000000 msgs, got %d", state.Msgs) @@ -994,12 +994,12 @@ func TestFileStoreStreamTruncate(t *testing.T) { subj, toStore := "foo", uint64(100) for i := uint64(1); i < tseq; i++ { - _, _, err := fs.StoreMsg(subj, nil, []byte("ok")) + _, _, err := fs.StoreMsg(subj, nil, []byte("ok"), 0) require_NoError(t, err) } subj = "bar" for i := tseq; i <= toStore; i++ { - _, _, err := fs.StoreMsg(subj, nil, []byte("ok")) + _, _, err := fs.StoreMsg(subj, nil, []byte("ok"), 0) require_NoError(t, err) } @@ -1082,7 +1082,7 @@ func TestFileStoreRemovePartialRecovery(t *testing.T) { subj, msg := "foo", []byte("Hello World") toStore := 100 for i := 0; i < toStore; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state := fs.State() if state.Msgs != uint64(toStore) { @@ -1125,7 +1125,7 @@ func TestFileStoreRemoveOutOfOrderRecovery(t *testing.T) { subj, msg := "foo", []byte("Hello World") toStore := 100 for i := 0; i < toStore; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state := fs.State() if state.Msgs != uint64(toStore) { @@ -1193,7 +1193,7 @@ func TestFileStoreAgeLimitRecovery(t *testing.T) { subj, msg := "foo", []byte("Hello World") toStore := 100 for i := 0; i < toStore; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state := fs.State() if state.Msgs != uint64(toStore) { @@ -1234,7 +1234,7 @@ func TestFileStoreBitRot(t *testing.T) { subj, msg := "foo", []byte("Hello World") toStore := 100 for i := 0; i < toStore; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state := fs.State() if state.Msgs != uint64(toStore) { @@ -1306,8 +1306,8 @@ func TestFileStoreEraseMsg(t *testing.T) { defer fs.Stop() subj, msg := "foo", []byte("Hello World") - fs.StoreMsg(subj, nil, msg) - fs.StoreMsg(subj, nil, msg) // To keep block from being deleted. + fs.StoreMsg(subj, nil, msg, 0) + fs.StoreMsg(subj, nil, msg, 0) // To keep block from being deleted. var smv StoreMsg sm, err := fs.LoadMsg(1, &smv) if err != nil { @@ -1367,7 +1367,7 @@ func TestFileStoreEraseAndNoIndexRecovery(t *testing.T) { subj, msg := "foo", []byte("Hello World") toStore := 100 for i := 0; i < toStore; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state := fs.State() if state.Msgs != uint64(toStore) { @@ -1513,7 +1513,7 @@ func TestFileStoreWriteAndReadSameBlock(t *testing.T) { subj, msg := "foo", []byte("Hello World!") for i := uint64(1); i <= 10; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) if _, err := fs.LoadMsg(i, nil); err != nil { t.Fatalf("Error loading %d: %v", i, err) } @@ -1534,7 +1534,7 @@ func TestFileStoreAndRetrieveMultiBlock(t *testing.T) { defer fs.Stop() for i := 0; i < 20; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state := fs.State() if state.Msgs != 20 { @@ -1566,7 +1566,7 @@ func TestFileStoreCollapseDmap(t *testing.T) { defer fs.Stop() for i := 0; i < 10; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state := fs.State() if state.Msgs != 10 { @@ -1639,7 +1639,7 @@ func TestFileStoreReadCache(t *testing.T) { totalBytes := uint64(toStore) * storedMsgSize for i := 0; i < toStore; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // Wait for cache to go to zero. @@ -1686,11 +1686,11 @@ func TestFileStorePartialCacheExpiration(t *testing.T) { require_NoError(t, err) defer fs.Stop() - fs.StoreMsg("foo", nil, []byte("msg1")) + fs.StoreMsg("foo", nil, []byte("msg1"), 0) // Should expire and be removed. time.Sleep(2 * cexp) - fs.StoreMsg("bar", nil, []byte("msg2")) + fs.StoreMsg("bar", nil, []byte("msg2"), 0) // Again wait for cache to expire. time.Sleep(2 * cexp) @@ -1711,7 +1711,7 @@ func TestFileStorePartialIndexes(t *testing.T) { toSend := 5 for i := 0; i < toSend; i++ { - fs.StoreMsg("foo", nil, []byte("ok-1")) + fs.StoreMsg("foo", nil, []byte("ok-1"), 0) } // Now wait til the cache expires, including the index. @@ -1735,7 +1735,7 @@ func TestFileStorePartialIndexes(t *testing.T) { // Create a partial cache by adding more msgs. for i := 0; i < toSend; i++ { - fs.StoreMsg("foo", nil, []byte("ok-2")) + fs.StoreMsg("foo", nil, []byte("ok-2"), 0) } // If we now load in a message in second half if we do not // detect idx is a partial correctly this will panic. @@ -1756,7 +1756,7 @@ func TestFileStoreSnapshot(t *testing.T) { toSend := 2233 for i := 0; i < toSend; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // Create a few consumers. @@ -2118,7 +2118,7 @@ func TestFileStoreWriteFailures(t *testing.T) { var lseq uint64 // msz about will be ~54 bytes, so if limit is 32k trying to send 1000 will fail at some point. for i := 1; i <= 1000; i++ { - if _, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { lseq = uint64(i) break } @@ -2160,7 +2160,7 @@ func TestFileStoreWriteFailures(t *testing.T) { // We should still fail here. for i := 1; i <= 100; i++ { - _, _, err = fs.StoreMsg(subj, nil, msg) + _, _, err = fs.StoreMsg(subj, nil, msg, 0) if err != nil { break } @@ -2178,7 +2178,7 @@ func TestFileStoreWriteFailures(t *testing.T) { // Check we will fail again in same spot. for i := 1; i <= 1000; i++ { - if _, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { if i != int(lseq) { t.Fatalf("Expected to fail after purge about the same spot, wanted %d got %d", lseq, i) } @@ -2217,7 +2217,7 @@ func TestFileStorePerf(t *testing.T) { start := time.Now() for i := 0; i < int(toStore); i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } fs.Stop() @@ -2340,7 +2340,7 @@ func TestFileStoreReadBackMsgPerf(t *testing.T) { start := time.Now() for i := 0; i < int(toStore); i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } tt := time.Since(start) @@ -2392,7 +2392,7 @@ func TestFileStoreStoreLimitRemovePerf(t *testing.T) { start := time.Now() for i := 0; i < int(toStore); i++ { - seq, _, err := fs.StoreMsg(subj, nil, msg) + seq, _, err := fs.StoreMsg(subj, nil, msg, 0) if err != nil { t.Fatalf("Unexpected error storing message: %v", err) } @@ -2436,7 +2436,7 @@ func TestFileStorePubPerfWithSmallBlkSize(t *testing.T) { start := time.Now() for i := 0; i < int(toStore); i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } fs.Stop() @@ -2721,7 +2721,7 @@ func TestFileStoreStreamStateDeleted(t *testing.T) { subj, toStore := "foo", uint64(10) for i := uint64(1); i <= toStore; i++ { msg := []byte(fmt.Sprintf("[%08d] Hello World!", i)) - if _, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Error storing msg: %v", err) } } @@ -2768,7 +2768,7 @@ func TestFileStoreStreamDeleteDirNotEmpty(t *testing.T) { subj, toStore := "foo", uint64(10) for i := uint64(1); i <= toStore; i++ { msg := []byte(fmt.Sprintf("[%08d] Hello World!", i)) - if _, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Error storing msg: %v", err) } } @@ -2866,10 +2866,10 @@ func TestFileStoreStreamDeleteCacheBug(t *testing.T) { subj, msg := "foo", []byte("Hello World") - if _, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Unexpected error: %v", err) } - if _, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Unexpected error: %v", err) } if _, err := fs.EraseMsg(1); err != nil { @@ -2894,7 +2894,7 @@ func TestFileStoreStreamFailToRollBug(t *testing.T) { // Make sure we properly roll underlying blocks. n, msg := 200, bytes.Repeat([]byte("ABC"), 33) // ~100bytes for i := 0; i < n; i++ { - if _, _, err := fs.StoreMsg("zzz", nil, msg); err != nil { + if _, _, err := fs.StoreMsg("zzz", nil, msg, 0); err != nil { t.Fatalf("Unexpected error: %v", err) } } @@ -2977,7 +2977,7 @@ func TestFileStoreExpireMsgsOnStart(t *testing.T) { loadMsgs := func(n int) { t.Helper() for i := 1; i <= n; i++ { - if _, _, err := fs.StoreMsg(fmt.Sprintf("orders.%d", i%10), nil, msg); err != nil { + if _, _, err := fs.StoreMsg(fmt.Sprintf("orders.%d", i%10), nil, msg, 0); err != nil { t.Fatalf("Unexpected error: %v", err) } } @@ -3150,7 +3150,7 @@ func TestFileStoreSparseCompaction(t *testing.T) { loadMsgs := func(n int) { t.Helper() for i := 1; i <= n; i++ { - if _, _, err := fs.StoreMsg(fmt.Sprintf("kv.%d", i%10), nil, msg); err != nil { + if _, _, err := fs.StoreMsg(fmt.Sprintf("kv.%d", i%10), nil, msg, 0); err != nil { t.Fatalf("Unexpected error: %v", err) } } @@ -3248,7 +3248,7 @@ func TestFileStoreSparseCompactionWithInteriorDeletes(t *testing.T) { defer fs.Stop() for i := 1; i <= 1000; i++ { - if _, _, err := fs.StoreMsg(fmt.Sprintf("kv.%d", i%10), nil, []byte("OK")); err != nil { + if _, _, err := fs.StoreMsg(fmt.Sprintf("kv.%d", i%10), nil, []byte("OK"), 0); err != nil { t.Fatalf("Unexpected error: %v", err) } } @@ -3292,10 +3292,10 @@ func TestFileStorePurgeExKeepOneBug(t *testing.T) { fill := bytes.Repeat([]byte("X"), 128) - fs.StoreMsg("A", nil, []byte("META")) - fs.StoreMsg("B", nil, fill) - fs.StoreMsg("A", nil, []byte("META")) - fs.StoreMsg("B", nil, fill) + fs.StoreMsg("A", nil, []byte("META"), 0) + fs.StoreMsg("B", nil, fill, 0) + fs.StoreMsg("A", nil, []byte("META"), 0) + fs.StoreMsg("B", nil, fill, 0) if fss := fs.FilteredState(1, "A"); fss.Msgs != 2 { t.Fatalf("Expected to find 2 `A` msgs, got %d", fss.Msgs) @@ -3321,9 +3321,9 @@ func TestFileStoreFilteredPendingBug(t *testing.T) { require_NoError(t, err) defer fs.Stop() - fs.StoreMsg("foo", nil, []byte("msg")) - fs.StoreMsg("bar", nil, []byte("msg")) - fs.StoreMsg("baz", nil, []byte("msg")) + fs.StoreMsg("foo", nil, []byte("msg"), 0) + fs.StoreMsg("bar", nil, []byte("msg"), 0) + fs.StoreMsg("baz", nil, []byte("msg"), 0) fs.mu.Lock() mb := fs.lmb @@ -3355,7 +3355,7 @@ func TestFileStoreFetchPerf(t *testing.T) { // Will create 25k msg blocks. n, subj, msg := 100_000, "zzz", bytes.Repeat([]byte("ABC"), 600) for i := 0; i < n; i++ { - if _, _, err := fs.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := fs.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Unexpected error: %v", err) } } @@ -3391,7 +3391,7 @@ func TestFileStoreCompactReclaimHeadSpace(t *testing.T) { // This gives us ~63 msgs in first and ~37 in second. n, subj := 100, "z" for i := 0; i < n; i++ { - _, _, err := fs.StoreMsg(subj, nil, msg) + _, _, err := fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } @@ -3465,7 +3465,7 @@ func TestFileStoreCompactReclaimHeadSpace(t *testing.T) { checkBlock(getBlock(0)) // Make sure we can write. - _, _, err = fs.StoreMsg(subj, nil, msg) + _, _, err = fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) checkNumBlocks(1) @@ -3482,7 +3482,7 @@ func TestFileStoreCompactReclaimHeadSpace(t *testing.T) { checkBlock(getBlock(0)) // Make sure we can write. - _, _, err = fs.StoreMsg(subj, nil, msg) + _, _, err = fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) }) } @@ -3510,7 +3510,7 @@ func TestFileStoreRememberLastMsgTime(t *testing.T) { fs = getFS() defer fs.Stop() - seq, ts, err := fs.StoreMsg("foo", nil, msg) + seq, ts, err := fs.StoreMsg("foo", nil, msg, 0) require_NoError(t, err) // We will test that last msg time survives from delete, purge and expires after restart. removed, err := fs.RemoveMsg(seq) @@ -3526,7 +3526,7 @@ func TestFileStoreRememberLastMsgTime(t *testing.T) { // Test that last time survived. require_True(t, lt == fs.State().LastTime) - seq, ts, err = fs.StoreMsg("foo", nil, msg) + seq, ts, err = fs.StoreMsg("foo", nil, msg, 0) require_NoError(t, err) var smv StoreMsg @@ -3541,9 +3541,9 @@ func TestFileStoreRememberLastMsgTime(t *testing.T) { lt = time.Unix(0, ts).UTC() require_True(t, lt == fs.State().LastTime) - _, _, err = fs.StoreMsg("foo", nil, msg) + _, _, err = fs.StoreMsg("foo", nil, msg, 0) require_NoError(t, err) - seq, ts, err = fs.StoreMsg("foo", nil, msg) + seq, ts, err = fs.StoreMsg("foo", nil, msg, 0) require_NoError(t, err) require_True(t, seq == 4) @@ -3564,9 +3564,9 @@ func TestFileStoreRememberLastMsgTime(t *testing.T) { require_True(t, lt == fs.State().LastTime) // Now make sure we retain the true last seq. - _, _, err = fs.StoreMsg("foo", nil, msg) + _, _, err = fs.StoreMsg("foo", nil, msg, 0) require_NoError(t, err) - seq, ts, err = fs.StoreMsg("foo", nil, msg) + seq, ts, err = fs.StoreMsg("foo", nil, msg, 0) require_NoError(t, err) require_True(t, seq == 6) @@ -3605,7 +3605,7 @@ func TestFileStoreRebuildStateDmapAccountingBug(t *testing.T) { defer fs.Stop() for i := 0; i < 100; i++ { - _, _, err = fs.StoreMsg("foo", nil, nil) + _, _, err = fs.StoreMsg("foo", nil, nil, 0) require_NoError(t, err) } // Delete 2-40. @@ -3658,15 +3658,15 @@ func TestFileStorePurgeExWithSubject(t *testing.T) { payload := make([]byte, 20) - _, _, err = fs.StoreMsg("foo.0", nil, payload) + _, _, err = fs.StoreMsg("foo.0", nil, payload, 0) require_NoError(t, err) total := 200 for i := 0; i < total; i++ { - _, _, err = fs.StoreMsg("foo.1", nil, payload) + _, _, err = fs.StoreMsg("foo.1", nil, payload, 0) require_NoError(t, err) } - _, _, err = fs.StoreMsg("foo.2", nil, []byte("xxxxxx")) + _, _, err = fs.StoreMsg("foo.2", nil, []byte("xxxxxx"), 0) require_NoError(t, err) // This should purge all "foo.1" @@ -3719,7 +3719,7 @@ func TestFileStoreShortIndexWriteBug(t *testing.T) { defer fs.Stop() for i := 0; i < 100; i++ { - _, _, err = fs.StoreMsg("foo", nil, nil) + _, _, err = fs.StoreMsg("foo", nil, nil, 0) require_NoError(t, err) } // Wait til messages all go away. @@ -3754,7 +3754,7 @@ func TestFileStoreDoubleCompactWithWriteInBetweenEncryptedBug(t *testing.T) { subj, msg := "foo", []byte("ouch") for i := 0; i < 5; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } _, err = fs.Compact(5) require_NoError(t, err) @@ -3763,7 +3763,7 @@ func TestFileStoreDoubleCompactWithWriteInBetweenEncryptedBug(t *testing.T) { t.Fatalf("Expected last sequence to be 5 but got %d", state.LastSeq) } for i := 0; i < 5; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } _, err = fs.Compact(10) require_NoError(t, err) @@ -3788,7 +3788,7 @@ func TestFileStoreEncryptedKeepIndexNeedBekResetBug(t *testing.T) { subj, msg := "foo", []byte("ouch") for i := 0; i < 5; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // Want to go to 0. @@ -3802,7 +3802,7 @@ func TestFileStoreEncryptedKeepIndexNeedBekResetBug(t *testing.T) { // Now write additional messages. for i := 0; i < 5; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // Make sure the buffer is cleared. @@ -3849,7 +3849,7 @@ func TestFileStoreExpireSubjectMeta(t *testing.T) { ns := 100 for i := 1; i <= ns; i++ { subj := fmt.Sprintf("kv.%d", i) - _, _, err := fs.StoreMsg(subj, nil, []byte("value")) + _, _, err := fs.StoreMsg(subj, nil, []byte("value"), 0) require_NoError(t, err) } @@ -3899,13 +3899,13 @@ func TestFileStoreMaxMsgsPerSubject(t *testing.T) { ns := 100 for i := 1; i <= ns; i++ { subj := fmt.Sprintf("kv.%d", i) - _, _, err := fs.StoreMsg(subj, nil, []byte("value")) + _, _, err := fs.StoreMsg(subj, nil, []byte("value"), 0) require_NoError(t, err) } for i := 1; i <= ns; i++ { subj := fmt.Sprintf("kv.%d", i) - _, _, err := fs.StoreMsg(subj, nil, []byte("value")) + _, _, err := fs.StoreMsg(subj, nil, []byte("value"), 0) require_NoError(t, err) } @@ -3937,7 +3937,7 @@ func TestFileStoreMaxMsgsAndMaxMsgsPerSubject(t *testing.T) { for i := 1; i <= 101; i++ { subj := fmt.Sprintf("kv.%d", i) - _, _, err := fs.StoreMsg(subj, nil, []byte("value")) + _, _, err := fs.StoreMsg(subj, nil, []byte("value"), 0) if i == 101 { // The 101th iteration should fail because MaxMsgs is set to // 100 and the policy is DiscardNew. @@ -3949,7 +3949,7 @@ func TestFileStoreMaxMsgsAndMaxMsgsPerSubject(t *testing.T) { for i := 1; i <= 100; i++ { subj := fmt.Sprintf("kv.%d", i) - _, _, err := fs.StoreMsg(subj, nil, []byte("value")) + _, _, err := fs.StoreMsg(subj, nil, []byte("value"), 0) // All of these iterations should fail because MaxMsgsPer is set // to 1 and DiscardNewPer is set to true, forcing us to reject // cases where there is already a message on this subject. @@ -3977,12 +3977,12 @@ func TestFileStoreSubjectStateCacheExpiration(t *testing.T) { for i := 1; i <= 100; i++ { subj := fmt.Sprintf("kv.foo.%d", i) - _, _, err := fs.StoreMsg(subj, nil, []byte("value")) + _, _, err := fs.StoreMsg(subj, nil, []byte("value"), 0) require_NoError(t, err) } for i := 1; i <= 100; i++ { subj := fmt.Sprintf("kv.bar.%d", i) - _, _, err := fs.StoreMsg(subj, nil, []byte("value")) + _, _, err := fs.StoreMsg(subj, nil, []byte("value"), 0) require_NoError(t, err) } @@ -4008,7 +4008,7 @@ func TestFileStoreSubjectStateCacheExpiration(t *testing.T) { } // Now add one to end and check as well for non-wildcard. - _, _, err = fs.StoreMsg("kv.foo.1", nil, []byte("value22")) + _, _, err = fs.StoreMsg("kv.foo.1", nil, []byte("value22"), 0) require_NoError(t, err) if state := fs.State(); state.Msgs != 201 { @@ -4035,7 +4035,7 @@ func TestFileStoreEncrypted(t *testing.T) { subj, msg := "foo", []byte("aes ftw") for i := 0; i < 50; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } o, err := fs.ConsumerStore("o22", &ConsumerConfig{}) @@ -4081,7 +4081,7 @@ func TestFileStoreNoFSSWhenNoSubjects(t *testing.T) { n, msg := 100, []byte("raft state") for i := 0; i < n; i++ { - _, _, err := fs.StoreMsg(_EMPTY_, nil, msg) + _, _, err := fs.StoreMsg(_EMPTY_, nil, msg, 0) require_NoError(t, err) } @@ -4122,7 +4122,7 @@ func TestFileStoreNoFSSBugAfterRemoveFirst(t *testing.T) { n, msg := 100, bytes.Repeat([]byte("ZZZ"), 33) // ~100bytes for i := 0; i < n; i++ { subj := fmt.Sprintf("foo.bar.%d", i) - _, _, err := fs.StoreMsg(subj, nil, msg) + _, _, err := fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } @@ -4164,7 +4164,7 @@ func TestFileStoreNoFSSAfterRecover(t *testing.T) { n, msg := 100, []byte("no fss for you!") for i := 0; i < n; i++ { - _, _, err := fs.StoreMsg(_EMPTY_, nil, msg) + _, _, err := fs.StoreMsg(_EMPTY_, nil, msg, 0) require_NoError(t, err) } @@ -4199,7 +4199,7 @@ func TestFileStoreFSSCloseAndKeepOnExpireOnRecoverBug(t *testing.T) { require_NoError(t, err) defer fs.Stop() - _, _, err = fs.StoreMsg("foo", nil, nil) + _, _, err = fs.StoreMsg("foo", nil, nil, 0) require_NoError(t, err) fs.Stop() @@ -4228,11 +4228,11 @@ func TestFileStoreExpireOnRecoverSubjectAccounting(t *testing.T) { defer fs.Stop() // These are in first block. - fs.StoreMsg("A", nil, msg) - fs.StoreMsg("B", nil, msg) + fs.StoreMsg("A", nil, msg, 0) + fs.StoreMsg("B", nil, msg, 0) time.Sleep(ttl / 2) // This one in 2nd block. - fs.StoreMsg("C", nil, msg) + fs.StoreMsg("C", nil, msg, 0) fs.Stop() time.Sleep(ttl/2 + 10*time.Millisecond) @@ -4259,7 +4259,7 @@ func TestFileStoreFSSExpireNumPendingBug(t *testing.T) { // Let FSS meta expire. time.Sleep(2 * cexp) - _, _, err = fs.StoreMsg("KV.X", nil, []byte("Y")) + _, _, err = fs.StoreMsg("KV.X", nil, []byte("Y"), 0) require_NoError(t, err) if fss := fs.FilteredState(1, "KV.X"); fss.Msgs != 1 { @@ -4276,13 +4276,13 @@ func TestFileStoreFilteredFirstMatchingBug(t *testing.T) { require_NoError(t, err) defer fs.Stop() - _, _, err = fs.StoreMsg("foo.foo", nil, []byte("A")) + _, _, err = fs.StoreMsg("foo.foo", nil, []byte("A"), 0) require_NoError(t, err) - _, _, err = fs.StoreMsg("foo.foo", nil, []byte("B")) + _, _, err = fs.StoreMsg("foo.foo", nil, []byte("B"), 0) require_NoError(t, err) - _, _, err = fs.StoreMsg("foo.foo", nil, []byte("C")) + _, _, err = fs.StoreMsg("foo.foo", nil, []byte("C"), 0) require_NoError(t, err) fs.mu.RLock() @@ -4300,7 +4300,7 @@ func TestFileStoreFilteredFirstMatchingBug(t *testing.T) { mb.mu.Unlock() // Now add in a different subject. - _, _, err = fs.StoreMsg("foo.bar", nil, []byte("X")) + _, _, err = fs.StoreMsg("foo.bar", nil, []byte("X"), 0) require_NoError(t, err) // Now see if a filtered load would incorrectly succeed. @@ -4318,10 +4318,10 @@ func TestFileStoreOutOfSpaceRebuildState(t *testing.T) { require_NoError(t, err) defer fs.Stop() - _, _, err = fs.StoreMsg("foo", nil, []byte("A")) + _, _, err = fs.StoreMsg("foo", nil, []byte("A"), 0) require_NoError(t, err) - _, _, err = fs.StoreMsg("bar", nil, []byte("B")) + _, _, err = fs.StoreMsg("bar", nil, []byte("B"), 0) require_NoError(t, err) // Grab state. @@ -4337,7 +4337,7 @@ func TestFileStoreOutOfSpaceRebuildState(t *testing.T) { mb.mockWriteErr = true mb.mu.Unlock() - _, _, err = fs.StoreMsg("baz", nil, []byte("C")) + _, _, err = fs.StoreMsg("baz", nil, []byte("C"), 0) require_Error(t, err, errors.New("mock write error")) nstate := fs.State() @@ -4362,14 +4362,14 @@ func TestFileStoreRebuildStateProperlyWithMaxMsgsPerSubject(t *testing.T) { defer fs.Stop() // Send one to baz at beginning. - _, _, err = fs.StoreMsg("baz", nil, nil) + _, _, err = fs.StoreMsg("baz", nil, nil, 0) require_NoError(t, err) ns := 1000 for i := 1; i <= ns; i++ { - _, _, err := fs.StoreMsg("foo", nil, nil) + _, _, err := fs.StoreMsg("foo", nil, nil, 0) require_NoError(t, err) - _, _, err = fs.StoreMsg("bar", nil, nil) + _, _, err = fs.StoreMsg("bar", nil, nil, 0) require_NoError(t, err) } @@ -4404,7 +4404,7 @@ func TestFileStoreUpdateMaxMsgsPerSubject(t *testing.T) { numStored := 22 for i := 0; i < numStored; i++ { - _, _, err = fs.StoreMsg("foo", nil, nil) + _, _, err = fs.StoreMsg("foo", nil, nil, 0) require_NoError(t, err) } @@ -4440,15 +4440,15 @@ func TestFileStoreBadFirstAndFailedExpireAfterRestart(t *testing.T) { // These are all instant and will expire after 1 sec. start := time.Now() for i := 0; i < 7; i++ { - _, _, err := fs.StoreMsg(subj, nil, msg) + _, _, err := fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } // Put two more after a delay. time.Sleep(1500 * time.Millisecond) - seq, _, err := fs.StoreMsg(subj, nil, msg) + seq, _, err := fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) - _, _, err = fs.StoreMsg(subj, nil, msg) + _, _, err = fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) // Make sure that sequence 8 is first in second block, and break test if that is not true. @@ -4498,7 +4498,7 @@ func TestFileStoreCompactAllWithDanglingLMB(t *testing.T) { subj, msg := "foo", []byte("ZZ") for i := 0; i < 100; i++ { - _, _, err := fs.StoreMsg(subj, nil, msg) + _, _, err := fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } @@ -4507,7 +4507,7 @@ func TestFileStoreCompactAllWithDanglingLMB(t *testing.T) { require_NoError(t, err) require_True(t, purged == 99) - _, _, err = fs.StoreMsg(subj, nil, msg) + _, _, err = fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) }) } @@ -4523,7 +4523,7 @@ func TestFileStoreStateWithBlkFirstDeleted(t *testing.T) { subj, msg := "foo", []byte("Hello World") toStore := 500 for i := 0; i < toStore; i++ { - _, _, err := fs.StoreMsg(subj, nil, msg) + _, _, err := fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } @@ -4561,7 +4561,7 @@ func TestFileStoreMsgBlkFailOnKernelFaultLostDataReporting(t *testing.T) { subj, msg := "foo", []byte("Hello World") toStore := 500 for i := 0; i < toStore; i++ { - _, _, err := fs.StoreMsg(subj, nil, msg) + _, _, err := fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } @@ -4661,7 +4661,7 @@ func TestFileStoreAllFilteredStateWithDeleted(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 0; i < 100; i++ { - _, _, err := fs.StoreMsg(subj, nil, msg) + _, _, err := fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } @@ -4711,7 +4711,7 @@ func TestFileStoreStreamTruncateResetMultiBlock(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 0; i < 1000; i++ { - _, _, err := fs.StoreMsg(subj, nil, msg) + _, _, err := fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } fs.syncBlocks() @@ -4730,7 +4730,7 @@ func TestFileStoreStreamTruncateResetMultiBlock(t *testing.T) { require_Equal(t, state.NumDeleted, 0) for i := 0; i < 1000; i++ { - _, _, err := fs.StoreMsg(subj, nil, msg) + _, _, err := fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } fs.syncBlocks() @@ -4755,7 +4755,7 @@ func TestFileStoreStreamCompactMultiBlockSubjectInfo(t *testing.T) { for i := 0; i < 1000; i++ { subj := fmt.Sprintf("foo.%d", i) - _, _, err := fs.StoreMsg(subj, nil, []byte("Hello World")) + _, _, err := fs.StoreMsg(subj, nil, []byte("Hello World"), 0) require_NoError(t, err) } require_True(t, fs.numMsgBlocks() == 500) @@ -4797,7 +4797,7 @@ func TestFileStoreSubjectsTotals(t *testing.T) { subj := fmt.Sprintf("%s.%d", ft, dt) m[dt]++ - _, _, err := fs.StoreMsg(subj, nil, []byte("Hello World")) + _, _, err := fs.StoreMsg(subj, nil, []byte("Hello World"), 0) require_NoError(t, err) } @@ -4900,7 +4900,7 @@ func TestFileStoreNumPendingLargeNumBlks(t *testing.T) { numMsgs := 10_000 for i := 0; i < numMsgs; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } start := time.Now() @@ -4942,11 +4942,11 @@ func TestFileStoreSkipMsgAndNumBlocks(t *testing.T) { subj, msg := "zzz", bytes.Repeat([]byte("X"), 100) numMsgs := 10_000 - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) for i := 0; i < numMsgs; i++ { fs.SkipMsg() } - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) require_True(t, fs.numMsgBlocks() == 2) } @@ -4961,7 +4961,7 @@ func TestFileStoreRestoreEncryptedWithNoKeyFuncFails(t *testing.T) { subj, msg := "zzz", bytes.Repeat([]byte("X"), 100) numMsgs := 100 for i := 0; i < numMsgs; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } fs.Stop() @@ -4978,13 +4978,13 @@ func TestFileStoreInitialFirstSeq(t *testing.T) { require_NoError(t, err) defer fs.Stop() - seq, _, err := fs.StoreMsg("A", nil, []byte("OK")) + seq, _, err := fs.StoreMsg("A", nil, []byte("OK"), 0) require_NoError(t, err) if seq != 1000 { t.Fatalf("Message should have been sequence 1000 but was %d", seq) } - seq, _, err = fs.StoreMsg("B", nil, []byte("OK")) + seq, _, err = fs.StoreMsg("B", nil, []byte("OK"), 0) require_NoError(t, err) if seq != 1001 { t.Fatalf("Message should have been sequence 1001 but was %d", seq) @@ -5008,9 +5008,9 @@ func TestFileStoreRecaluclateFirstForSubjBug(t *testing.T) { require_NoError(t, err) defer fs.Stop() - fs.StoreMsg("foo", nil, nil) // 1 - fs.StoreMsg("bar", nil, nil) // 2 - fs.StoreMsg("foo", nil, nil) // 3 + fs.StoreMsg("foo", nil, nil, 0) // 1 + fs.StoreMsg("bar", nil, nil, 0) // 2 + fs.StoreMsg("foo", nil, nil, 0) // 3 // Now remove first 2.. fs.RemoveMsg(1) @@ -5043,8 +5043,8 @@ func TestFileStoreKeepWithDeletedMsgsBug(t *testing.T) { msg := bytes.Repeat([]byte("A"), 19) for i := 0; i < 5; i++ { - fs.StoreMsg("A", nil, msg) - fs.StoreMsg("B", nil, msg) + fs.StoreMsg("A", nil, msg, 0) + fs.StoreMsg("B", nil, msg, 0) } n, err := fs.PurgeEx("A", 0, 0) @@ -5067,8 +5067,8 @@ func TestFileStoreRestartWithExpireAndLockingBug(t *testing.T) { // 20 total msg := []byte("HELLO WORLD") for i := 0; i < 10; i++ { - fs.StoreMsg("A", nil, msg) - fs.StoreMsg("B", nil, msg) + fs.StoreMsg("A", nil, msg, 0) + fs.StoreMsg("B", nil, msg, 0) } fs.Stop() @@ -5094,7 +5094,7 @@ func TestFileStoreErrPartialLoad(t *testing.T) { put := func(num int) { for i := 0; i < num; i++ { - fs.StoreMsg("Z", nil, []byte("ZZZZZZZZZZZZZ")) + fs.StoreMsg("Z", nil, []byte("ZZZZZZZZZZZZZ"), 0) } } @@ -5163,7 +5163,7 @@ func TestFileStoreErrPartialLoadOnSyncClose(t *testing.T) { // Load up half the block. for _, subj := range []string{"A", "B", "C", "D", "E"} { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // Now simulate the sync timer closing the last block. @@ -5177,7 +5177,7 @@ func TestFileStoreErrPartialLoadOnSyncClose(t *testing.T) { lmb.dirtyCloseWithRemove(false) lmb.mu.Unlock() - fs.StoreMsg("Z", nil, msg) + fs.StoreMsg("Z", nil, msg, 0) _, err = fs.LoadMsg(1, nil) require_NoError(t, err) } @@ -5201,7 +5201,7 @@ func TestFileStoreSyncIntervals(t *testing.T) { } checkSyncFlag(false) - fs.StoreMsg("Z", nil, []byte("hello")) + fs.StoreMsg("Z", nil, []byte("hello"), 0) checkSyncFlag(true) time.Sleep(400 * time.Millisecond) checkSyncFlag(false) @@ -5215,7 +5215,7 @@ func TestFileStoreSyncIntervals(t *testing.T) { defer fs.Stop() checkSyncFlag(false) - fs.StoreMsg("Z", nil, []byte("hello")) + fs.StoreMsg("Z", nil, []byte("hello"), 0) checkSyncFlag(false) } @@ -5230,7 +5230,7 @@ func TestFileStoreRecalcFirstSequenceBug(t *testing.T) { msg := bytes.Repeat([]byte("A"), 22) for _, subj := range []string{"A", "A", "B", "B"} { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // Make sure the buffer is cleared. clearLMBCache := func() { @@ -5245,7 +5245,7 @@ func TestFileStoreRecalcFirstSequenceBug(t *testing.T) { clearLMBCache() // Do first here. - fs.StoreMsg("A", nil, msg) + fs.StoreMsg("A", nil, msg, 0) var wg sync.WaitGroup start := make(chan bool) @@ -5265,7 +5265,7 @@ func TestFileStoreRecalcFirstSequenceBug(t *testing.T) { defer wg.Done() <-start for i := 0; i < 1_000; i++ { - fs.StoreMsg("A", nil, msg) + fs.StoreMsg("A", nil, msg, 0) } }() @@ -5292,8 +5292,8 @@ func TestFileStoreFullStateBasics(t *testing.T) { msgZ := bytes.Repeat([]byte("Z"), msgLen) // Send 2 msgs and stop, check for presence of our full state file. - fs.StoreMsg(subj, nil, msgA) - fs.StoreMsg(subj, nil, msgZ) + fs.StoreMsg(subj, nil, msgA, 0) + fs.StoreMsg(subj, nil, msgZ, 0) require_True(t, fs.numMsgBlocks() == 1) // Make sure there is a full state file after we do a stop. @@ -5338,7 +5338,7 @@ func TestFileStoreFullStateBasics(t *testing.T) { require_True(t, bytes.Equal(sm.msg, msgZ)) // Now add in 1 more here to split the lmb. - fs.StoreMsg(subj, nil, msgZ) + fs.StoreMsg(subj, nil, msgZ, 0) // Now stop the filestore and replace the old stream state and make sure we recover correctly. fs.Stop() @@ -5352,7 +5352,7 @@ func TestFileStoreFullStateBasics(t *testing.T) { defer fs.Stop() // Add in one more. - fs.StoreMsg(subj, nil, msgZ) + fs.StoreMsg(subj, nil, msgZ, 0) fs.Stop() // Put old stream state back with only 3. @@ -5381,7 +5381,7 @@ func TestFileStoreFullStateBasics(t *testing.T) { require_Equal(t, psi.lblk, 2) // Store 1 more - fs.StoreMsg(subj, nil, msgA) + fs.StoreMsg(subj, nil, msgA, 0) fs.Stop() // Put old stream state back with only 3. err = os.WriteFile(sfile, buf, defaultFilePerms) @@ -5421,7 +5421,7 @@ func TestFileStoreFullStatePurge(t *testing.T) { // Should be 2 per block, so 5 blocks. for i := 0; i < 10; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } n, err := fs.Purge() require_NoError(t, err) @@ -5440,8 +5440,8 @@ func TestFileStoreFullStatePurge(t *testing.T) { // Add in more 10 more total, some B some C. for i := 0; i < 5; i++ { - fs.StoreMsg("B", nil, msg) - fs.StoreMsg("C", nil, msg) + fs.StoreMsg("B", nil, msg, 0) + fs.StoreMsg("C", nil, msg, 0) } n, err = fs.PurgeEx("B", 0, 0) @@ -5520,8 +5520,8 @@ func TestFileStoreFullStateTestUserRemoveWAL(t *testing.T) { msgZ := bytes.Repeat([]byte("Z"), msgLen) // Store 2 msgs and delete first. - fs.StoreMsg("A", nil, msgA) - fs.StoreMsg("Z", nil, msgZ) + fs.StoreMsg("A", nil, msgA, 0) + fs.StoreMsg("Z", nil, msgZ, 0) fs.RemoveMsg(1) // Check we can load things properly since the block will have a tombstone now for seq 1. @@ -5555,8 +5555,8 @@ func TestFileStoreFullStateTestUserRemoveWAL(t *testing.T) { require_True(t, !state.FirstTime.IsZero()) // Store 2 more msgs and delete 2 & 4. - fs.StoreMsg("A", nil, msgA) - fs.StoreMsg("Z", nil, msgZ) + fs.StoreMsg("A", nil, msgA, 0) + fs.StoreMsg("Z", nil, msgZ, 0) fs.RemoveMsg(2) fs.RemoveMsg(4) @@ -5620,7 +5620,7 @@ func TestFileStoreFullStateTestSysRemovals(t *testing.T) { msg := bytes.Repeat([]byte("A"), msgLen) for _, subj := range []string{"A", "B", "A", "B"} { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state := fs.State() @@ -5639,7 +5639,7 @@ func TestFileStoreFullStateTestSysRemovals(t *testing.T) { } for _, subj := range []string{"C", "D", "E", "F", "G", "H", "I", "J"} { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } state = fs.State() @@ -5658,7 +5658,7 @@ func TestFileStoreFullStateTestSysRemovals(t *testing.T) { } // Goes over limit - fs.StoreMsg("ZZZ", nil, msg) + fs.StoreMsg("ZZZ", nil, msg, 0) state = fs.State() require_Equal(t, state.Msgs, 10) @@ -5698,7 +5698,7 @@ func TestFileStoreSelectBlockWithFirstSeqRemovals(t *testing.T) { // We need over 32 blocks to kick in binary search. So 32*2+1 (65) msgs to get 33 blocks. for i := 0; i < 32*2+1; i++ { subj := string(subjects[i]) - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } require_Equal(t, fs.numMsgBlocks(), 33) @@ -5706,7 +5706,7 @@ func TestFileStoreSelectBlockWithFirstSeqRemovals(t *testing.T) { // Want to do this via system removes, not user initiated moves. for i := 0; i < len(subjects); i += 2 { subj := string(subjects[i]) - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } var ss StreamState @@ -5796,7 +5796,7 @@ func TestFileStoreMsgBlockCompactionAndHoles(t *testing.T) { msg := bytes.Repeat([]byte("Z"), 1024) for _, subj := range []string{"A", "B", "C", "D", "E", "F", "G", "H", "I", "J"} { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // Leave first one but delete the rest. for seq := uint64(2); seq < 10; seq++ { @@ -5826,7 +5826,7 @@ func TestFileStoreRemoveLastNoDoubleTombstones(t *testing.T) { require_NoError(t, err) defer fs.Stop() - fs.StoreMsg("A", nil, []byte("hello")) + fs.StoreMsg("A", nil, []byte("hello"), 0) fs.mu.Lock() fs.removeMsgViaLimits(1) fs.mu.Unlock() @@ -5855,8 +5855,8 @@ func TestFileStoreFullStateMultiBlockPastWAL(t *testing.T) { msgZ := bytes.Repeat([]byte("Z"), msgLen) // Store 2 msgs - fs.StoreMsg("A", nil, msgA) - fs.StoreMsg("B", nil, msgZ) + fs.StoreMsg("A", nil, msgA, 0) + fs.StoreMsg("B", nil, msgZ, 0) require_Equal(t, fs.numMsgBlocks(), 1) fs.Stop() @@ -5870,12 +5870,12 @@ func TestFileStoreFullStateMultiBlockPastWAL(t *testing.T) { defer fs.Stop() // Store 6 more msgs. - fs.StoreMsg("C", nil, msgA) - fs.StoreMsg("D", nil, msgZ) - fs.StoreMsg("E", nil, msgA) - fs.StoreMsg("F", nil, msgZ) - fs.StoreMsg("G", nil, msgA) - fs.StoreMsg("H", nil, msgZ) + fs.StoreMsg("C", nil, msgA, 0) + fs.StoreMsg("D", nil, msgZ, 0) + fs.StoreMsg("E", nil, msgA, 0) + fs.StoreMsg("F", nil, msgZ, 0) + fs.StoreMsg("G", nil, msgA, 0) + fs.StoreMsg("H", nil, msgZ, 0) require_Equal(t, fs.numMsgBlocks(), 4) state := fs.State() fs.Stop() @@ -5912,11 +5912,11 @@ func TestFileStoreFullStateMidBlockPastWAL(t *testing.T) { msg := bytes.Repeat([]byte("Z"), 19) // Store 5 msgs - fs.StoreMsg("A", nil, msg) - fs.StoreMsg("B", nil, msg) - fs.StoreMsg("C", nil, msg) - fs.StoreMsg("D", nil, msg) - fs.StoreMsg("E", nil, msg) + fs.StoreMsg("A", nil, msg, 0) + fs.StoreMsg("B", nil, msg, 0) + fs.StoreMsg("C", nil, msg, 0) + fs.StoreMsg("D", nil, msg, 0) + fs.StoreMsg("E", nil, msg, 0) require_Equal(t, fs.numMsgBlocks(), 1) fs.Stop() @@ -5930,11 +5930,11 @@ func TestFileStoreFullStateMidBlockPastWAL(t *testing.T) { defer fs.Stop() // Store 5 more messages, then remove seq 2, "B". - fs.StoreMsg("F", nil, msg) - fs.StoreMsg("G", nil, msg) - fs.StoreMsg("H", nil, msg) - fs.StoreMsg("I", nil, msg) - fs.StoreMsg("J", nil, msg) + fs.StoreMsg("F", nil, msg, 0) + fs.StoreMsg("G", nil, msg, 0) + fs.StoreMsg("H", nil, msg, 0) + fs.StoreMsg("I", nil, msg, 0) + fs.StoreMsg("J", nil, msg, 0) fs.RemoveMsg(2) require_Equal(t, fs.numMsgBlocks(), 1) @@ -5973,7 +5973,7 @@ func TestFileStoreCompactingBlocksOnSync(t *testing.T) { msg := bytes.Repeat([]byte("Z"), 19) subjects := "ABCDEFGHIJKLMNOPQRST" for _, subj := range subjects { - fs.StoreMsg(string(subj), nil, msg) + fs.StoreMsg(string(subj), nil, msg, 0) } require_Equal(t, fs.numMsgBlocks(), 1) total, reported, err := fs.Utilization() @@ -5984,7 +5984,7 @@ func TestFileStoreCompactingBlocksOnSync(t *testing.T) { // Now start removing, since we are small this should not kick in any inline logic. // Remove all interior messages, leave 1 and 20. So write B-S for i := 1; i < 19; i++ { - fs.StoreMsg(string(subjects[i]), nil, msg) + fs.StoreMsg(string(subjects[i]), nil, msg, 0) } require_Equal(t, fs.numMsgBlocks(), 2) @@ -6029,7 +6029,7 @@ func TestFileStoreCompactAndPSIMWhenDeletingBlocks(t *testing.T) { // Add in 10 As for i := 0; i < 10; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } require_Equal(t, fs.numMsgBlocks(), 4) @@ -6079,7 +6079,7 @@ func TestFileStoreTrackSubjLenForPSIM(t *testing.T) { continue } smap[subj] = len(subj) - fs.StoreMsg(subj, nil, nil) + fs.StoreMsg(subj, nil, nil, 0) } check := func() { @@ -6150,7 +6150,7 @@ func TestFileStoreLargeFullStatePSIM(t *testing.T) { } } subj := b.String() - fs.StoreMsg(subj, nil, nil) + fs.StoreMsg(subj, nil, nil, 0) } fs.Stop() } @@ -6165,7 +6165,7 @@ func TestFileStoreLargeFullStateMetaCleanup(t *testing.T) { subj, msg := "foo.bar.baz", bytes.Repeat([]byte("ABC"), 33) // ~100bytes for i := 0; i < 1000; i++ { - fs.StoreMsg(subj, nil, nil) + fs.StoreMsg(subj, nil, nil, 0) } fs.Stop() @@ -6202,7 +6202,7 @@ func TestFileStoreIndexDBExistsAfterShutdown(t *testing.T) { subj := "foo.bar.baz" for i := 0; i < 1000; i++ { - fs.StoreMsg(subj, nil, nil) + fs.StoreMsg(subj, nil, nil, 0) } idxFile := filepath.Join(sd, msgDir, streamStreamStateFile) @@ -6238,7 +6238,7 @@ func TestFileStoreSubjectCorruption(t *testing.T) { for i := 0; i < 10_000; i++ { subj := fmt.Sprintf("foo.%d", rand.Intn(numSubjects)+1) msg := msgs[rand.Intn(len(msgs))] - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } fs.Stop() @@ -6272,7 +6272,7 @@ func TestFileStoreNumPendingLastBySubject(t *testing.T) { msg := bytes.Repeat([]byte("ABC"), 25) for i := 1; i <= 1000; i++ { subj := fmt.Sprintf("foo.%d.%d", rand.Intn(numSubjects)+1, i) - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // Each block has ~8 msgs. require_True(t, fs.numMsgBlocks() > 100) @@ -6325,8 +6325,8 @@ func TestFileStoreCorruptPSIMOnDisk(t *testing.T) { require_NoError(t, err) defer fs.Stop() - fs.StoreMsg("foo.bar", nil, []byte("ABC")) - fs.StoreMsg("foo.baz", nil, []byte("XYZ")) + fs.StoreMsg("foo.bar", nil, []byte("ABC"), 0) + fs.StoreMsg("foo.baz", nil, []byte("XYZ"), 0) // Force bad subject. fs.mu.Lock() @@ -6365,8 +6365,8 @@ func TestFileStorePurgeExBufPool(t *testing.T) { msg := bytes.Repeat([]byte("ABC"), 33) // ~100bytes for i := 0; i < 1000; i++ { - fs.StoreMsg("foo.foo", nil, msg) - fs.StoreMsg("foo.bar", nil, msg) + fs.StoreMsg("foo.foo", nil, msg, 0) + fs.StoreMsg("foo.bar", nil, msg, 0) } p, err := fs.PurgeEx("foo.bar", 1, 0) @@ -6399,11 +6399,11 @@ func TestFileStoreFSSMeta(t *testing.T) { msg := bytes.Repeat([]byte("Z"), 19) // Should leave us with |A-Z| |Z-Z| |Z-Z| |Z-A| - fs.StoreMsg("A", nil, msg) + fs.StoreMsg("A", nil, msg, 0) for i := 0; i < 6; i++ { - fs.StoreMsg("Z", nil, msg) + fs.StoreMsg("Z", nil, msg, 0) } - fs.StoreMsg("A", nil, msg) + fs.StoreMsg("A", nil, msg, 0) // Let cache's expire before PurgeEx which will load them back in. time.Sleep(250 * time.Millisecond) @@ -6453,7 +6453,7 @@ func TestFileStoreExpireCacheOnLinearWalk(t *testing.T) { // Store 10 messages, so 5 blocks. for i := 0; i < 10; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // Let them all expire. This way we load as we walk and can test that we expire all blocks without // needing to worry about last write times blocking forced expiration. @@ -6543,7 +6543,7 @@ func TestFileStoreSkipMsgs(t *testing.T) { require_NoError(t, err) defer fs.Stop() - fs.StoreMsg("foo", nil, nil) + fs.StoreMsg("foo", nil, nil, 0) err = fs.SkipMsgs(2, 10) require_NoError(t, err) state = fs.State() @@ -6573,13 +6573,13 @@ func TestFileStoreOptimizeFirstLoadNextMsgWithSequenceZero(t *testing.T) { msg := bytes.Repeat([]byte("ZZZ"), 33) // ~100bytes for i := 0; i < 5000; i++ { - fs.StoreMsg("foo.A", nil, msg) + fs.StoreMsg("foo.A", nil, msg, 0) } // This will create alot of blocks, ~167. // Just used to check that we do not load these in when searching. // Now add in 10 for foo.bar at the end. for i := 0; i < 10; i++ { - fs.StoreMsg("foo.B", nil, msg) + fs.StoreMsg("foo.B", nil, msg, 0) } // The bug would not be visible on running server per se since we would have had fss loaded // and that sticks around a bit longer, we would use that to skip over the early blocks. So stop @@ -6613,7 +6613,7 @@ func TestFileStoreWriteFullStateHighSubjectCardinality(t *testing.T) { for i := 0; i < 1_000_000; i++ { subj := fmt.Sprintf("subj_%d", i) - _, _, err := fs.StoreMsg(subj, nil, msg) + _, _, err := fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } @@ -6629,11 +6629,11 @@ func TestFileStoreEraseMsgWithDbitSlots(t *testing.T) { require_NoError(t, err) defer fs.Stop() - fs.StoreMsg("foo", nil, []byte("abd")) + fs.StoreMsg("foo", nil, []byte("abd"), 0) for i := 0; i < 10; i++ { fs.SkipMsg() } - fs.StoreMsg("foo", nil, []byte("abd")) + fs.StoreMsg("foo", nil, []byte("abd"), 0) // Now grab that first block and compact away the skips which will // introduce dbits into our idx. fs.mu.RLock() @@ -6656,8 +6656,8 @@ func TestFileStoreEraseMsgWithAllTrailingDbitSlots(t *testing.T) { require_NoError(t, err) defer fs.Stop() - fs.StoreMsg("foo", nil, []byte("abc")) - fs.StoreMsg("foo", nil, []byte("abcdefg")) + fs.StoreMsg("foo", nil, []byte("abc"), 0) + fs.StoreMsg("foo", nil, []byte("abcdefg"), 0) for i := 0; i < 10; i++ { fs.SkipMsg() @@ -6686,14 +6686,14 @@ func TestFileStoreMultiLastSeqs(t *testing.T) { msg := []byte("abc") for i := 0; i < 33; i++ { - fs.StoreMsg("foo.foo", nil, msg) - fs.StoreMsg("foo.bar", nil, msg) - fs.StoreMsg("foo.baz", nil, msg) + fs.StoreMsg("foo.foo", nil, msg, 0) + fs.StoreMsg("foo.bar", nil, msg, 0) + fs.StoreMsg("foo.baz", nil, msg, 0) } for i := 0; i < 33; i++ { - fs.StoreMsg("bar.foo", nil, msg) - fs.StoreMsg("bar.bar", nil, msg) - fs.StoreMsg("bar.baz", nil, msg) + fs.StoreMsg("bar.foo", nil, msg, 0) + fs.StoreMsg("bar.bar", nil, msg, 0) + fs.StoreMsg("bar.baz", nil, msg, 0) } checkResults := func(seqs, expected []uint64) { @@ -6770,7 +6770,7 @@ func TestFileStoreMultiLastSeqsMaxAllowed(t *testing.T) { msg := []byte("abc") for i := 1; i <= 100; i++ { - fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) } // Test that if we specify maxAllowed that we get the correct error. seqs, err := fs.MultiLastSeqs([]string{"foo.*"}, 0, 10) @@ -6789,7 +6789,7 @@ func TestFileStoreMsgBlockFirstAndLastSeqCorrupt(t *testing.T) { msg := []byte("abc") for i := 1; i <= 10; i++ { - fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) } fs.Purge() @@ -6820,7 +6820,7 @@ func TestFileStoreWriteFullStateAfterPurgeEx(t *testing.T) { msg := []byte("abc") for i := 1; i <= 10; i++ { - fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) } fs.RemoveMsg(8) fs.RemoveMsg(9) @@ -6851,7 +6851,7 @@ func TestFileStoreFSSExpire(t *testing.T) { msg := []byte("abc") for i := 1; i <= 1000; i++ { - fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) } // Flush fss by hand, cache should be flushed as well. fs.mu.RLock() @@ -6862,11 +6862,11 @@ func TestFileStoreFSSExpire(t *testing.T) { } fs.mu.RUnlock() - fs.StoreMsg("foo.11", nil, msg) + fs.StoreMsg("foo.11", nil, msg, 0) time.Sleep(900 * time.Millisecond) // This should keep fss alive in the first block.. // As well as cache itself due to remove activity. - fs.StoreMsg("foo.22", nil, msg) + fs.StoreMsg("foo.22", nil, msg, 0) time.Sleep(300 * time.Millisecond) // Check that fss and the cache are still loaded. fs.mu.RLock() @@ -6888,8 +6888,8 @@ func TestFileStoreFSSExpireNumPending(t *testing.T) { msg := []byte("abc") for i := 1; i <= 100_000; i++ { - fs.StoreMsg(fmt.Sprintf("foo.A.%d", i), nil, msg) - fs.StoreMsg(fmt.Sprintf("foo.B.%d", i), nil, msg) + fs.StoreMsg(fmt.Sprintf("foo.A.%d", i), nil, msg, 0) + fs.StoreMsg(fmt.Sprintf("foo.B.%d", i), nil, msg, 0) } // Flush fss by hand, cache should be flushed as well. fs.mu.RLock() @@ -6962,7 +6962,7 @@ func TestFileStoreRecoverWithRemovesAndNoIndexDB(t *testing.T) { msg := []byte("abc") for i := 1; i <= 10; i++ { - fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) } fs.RemoveMsg(1) fs.RemoveMsg(2) @@ -7060,10 +7060,10 @@ func TestFileStoreLoadLastWildcard(t *testing.T) { defer fs.Stop() msg := []byte("hello") - fs.StoreMsg("foo.22.baz", nil, msg) - fs.StoreMsg("foo.22.bar", nil, msg) + fs.StoreMsg("foo.22.baz", nil, msg, 0) + fs.StoreMsg("foo.22.bar", nil, msg, 0) for i := 0; i < 1000; i++ { - fs.StoreMsg("foo.11.foo", nil, msg) + fs.StoreMsg("foo.11.foo", nil, msg, 0) } // Make sure we remove fss since that would mask the problem that we walk @@ -7102,10 +7102,10 @@ func TestFileStoreLoadLastWildcardWithPresenceMultipleBlocks(t *testing.T) { defer fs.Stop() // Make sure we have "foo.222.bar" in multiple blocks to show bug. - fs.StoreMsg("foo.22.bar", nil, []byte("hello")) - fs.StoreMsg("foo.22.baz", nil, []byte("ok")) - fs.StoreMsg("foo.22.baz", nil, []byte("ok")) - fs.StoreMsg("foo.22.bar", nil, []byte("hello22")) + fs.StoreMsg("foo.22.bar", nil, []byte("hello"), 0) + fs.StoreMsg("foo.22.baz", nil, []byte("ok"), 0) + fs.StoreMsg("foo.22.baz", nil, []byte("ok"), 0) + fs.StoreMsg("foo.22.bar", nil, []byte("hello22"), 0) require_True(t, fs.numMsgBlocks() > 1) sm, err := fs.LoadLastMsg("foo.*.bar", nil) require_NoError(t, err) @@ -7124,13 +7124,13 @@ func TestFileStoreFilteredPendingPSIMFirstBlockUpdate(t *testing.T) { // When PSIM detects msgs == 1 will catch up, so msgs needs to be > 1. // Then create a huge block gap. msg := []byte("hello") - fs.StoreMsg("foo.baz", nil, msg) + fs.StoreMsg("foo.baz", nil, msg, 0) for i := 0; i < 1000; i++ { - fs.StoreMsg("foo.foo", nil, msg) + fs.StoreMsg("foo.foo", nil, msg, 0) } // Bookend with 2 more foo.baz - fs.StoreMsg("foo.baz", nil, msg) - fs.StoreMsg("foo.baz", nil, msg) + fs.StoreMsg("foo.baz", nil, msg, 0) + fs.StoreMsg("foo.baz", nil, msg, 0) // Now remove first one. removed, err := fs.RemoveMsg(1) require_NoError(t, err) @@ -7182,16 +7182,16 @@ func TestFileStoreWildcardFilteredPendingPSIMFirstBlockUpdate(t *testing.T) { // When PSIM detects msgs == 1 will catch up, so msgs needs to be > 1. // Then create a huge block gap. msg := []byte("hello") - fs.StoreMsg("foo.22.baz", nil, msg) - fs.StoreMsg("foo.22.bar", nil, msg) + fs.StoreMsg("foo.22.baz", nil, msg, 0) + fs.StoreMsg("foo.22.bar", nil, msg, 0) for i := 0; i < 1000; i++ { - fs.StoreMsg("foo.1.foo", nil, msg) + fs.StoreMsg("foo.1.foo", nil, msg, 0) } // Bookend with 3 more, two foo.baz and two foo.bar. - fs.StoreMsg("foo.22.baz", nil, msg) - fs.StoreMsg("foo.22.baz", nil, msg) - fs.StoreMsg("foo.22.bar", nil, msg) - fs.StoreMsg("foo.22.bar", nil, msg) + fs.StoreMsg("foo.22.baz", nil, msg, 0) + fs.StoreMsg("foo.22.baz", nil, msg, 0) + fs.StoreMsg("foo.22.bar", nil, msg, 0) + fs.StoreMsg("foo.22.bar", nil, msg, 0) // Now remove first one for foo.bar and foo.baz. removed, err := fs.RemoveMsg(1) @@ -7272,8 +7272,8 @@ func TestFileStoreFilteredPendingPSIMFirstBlockUpdateNextBlock(t *testing.T) { msg := []byte("hello") // Create 4 blocks, each block holds 2 msgs for i := 0; i < 4; i++ { - fs.StoreMsg("foo.22.bar", nil, msg) - fs.StoreMsg("foo.22.baz", nil, msg) + fs.StoreMsg("foo.22.bar", nil, msg, 0) + fs.StoreMsg("foo.22.baz", nil, msg, 0) } require_Equal(t, fs.numMsgBlocks(), 4) @@ -7363,14 +7363,14 @@ func TestFileStoreLargeSparseMsgsDoNotLoadAfterLast(t *testing.T) { msg := []byte("hello") // Create 2 blocks with each, each block holds 2 msgs for i := 0; i < 2; i++ { - fs.StoreMsg("foo.22.bar", nil, msg) - fs.StoreMsg("foo.22.baz", nil, msg) + fs.StoreMsg("foo.22.bar", nil, msg, 0) + fs.StoreMsg("foo.22.baz", nil, msg, 0) } // Now create 8 more blocks with just baz. So no matches for these 8 blocks // for "foo.22.bar". for i := 0; i < 8; i++ { - fs.StoreMsg("foo.22.baz", nil, msg) - fs.StoreMsg("foo.22.baz", nil, msg) + fs.StoreMsg("foo.22.baz", nil, msg, 0) + fs.StoreMsg("foo.22.baz", nil, msg, 0) } require_Equal(t, fs.numMsgBlocks(), 10) @@ -7414,14 +7414,14 @@ func TestFileStoreCheckSkipFirstBlockBug(t *testing.T) { msg := []byte("hello") - fs.StoreMsg("foo.BB.bar", nil, msg) - fs.StoreMsg("foo.BB.bar", nil, msg) - fs.StoreMsg("foo.AA.bar", nil, msg) + fs.StoreMsg("foo.BB.bar", nil, msg, 0) + fs.StoreMsg("foo.BB.bar", nil, msg, 0) + fs.StoreMsg("foo.AA.bar", nil, msg, 0) for i := 0; i < 5; i++ { - fs.StoreMsg("foo.BB.bar", nil, msg) + fs.StoreMsg("foo.BB.bar", nil, msg, 0) } - fs.StoreMsg("foo.AA.bar", nil, msg) - fs.StoreMsg("foo.AA.bar", nil, msg) + fs.StoreMsg("foo.AA.bar", nil, msg, 0) + fs.StoreMsg("foo.AA.bar", nil, msg, 0) // Should have created 4 blocks. // BB BB | AA BB | BB BB | BB BB | AA AA @@ -7450,7 +7450,7 @@ func TestFileStoreTombstoneRbytes(t *testing.T) { // So will fill one block and half of the other msg := []byte("hello") for i := 0; i < 34; i++ { - fs.StoreMsg("foo.22", nil, msg) + fs.StoreMsg("foo.22", nil, msg, 0) } require_True(t, fs.numMsgBlocks() > 1) // Now delete second half of first block which will place tombstones in second blk. @@ -7478,7 +7478,7 @@ func TestFileStoreMsgBlockShouldCompact(t *testing.T) { // 127 fit into a block. msg := bytes.Repeat([]byte("Z"), 64*1024) for i := 0; i < 190; i++ { - fs.StoreMsg("foo.22", nil, msg) + fs.StoreMsg("foo.22", nil, msg, 0) } require_True(t, fs.numMsgBlocks() > 1) // Now delete second half of first block which will place tombstones in second blk. @@ -7514,16 +7514,16 @@ func TestFileStoreCheckSkipFirstBlockNotLoadOldBlocks(t *testing.T) { msg := []byte("hello") - fs.StoreMsg("foo.BB.bar", nil, msg) - fs.StoreMsg("foo.AA.bar", nil, msg) + fs.StoreMsg("foo.BB.bar", nil, msg, 0) + fs.StoreMsg("foo.AA.bar", nil, msg, 0) for i := 0; i < 6; i++ { - fs.StoreMsg("foo.BB.bar", nil, msg) + fs.StoreMsg("foo.BB.bar", nil, msg, 0) } - fs.StoreMsg("foo.AA.bar", nil, msg) // Sequence 9 - fs.StoreMsg("foo.AA.bar", nil, msg) // Sequence 10 + fs.StoreMsg("foo.AA.bar", nil, msg, 0) // Sequence 9 + fs.StoreMsg("foo.AA.bar", nil, msg, 0) // Sequence 10 for i := 0; i < 4; i++ { - fs.StoreMsg("foo.BB.bar", nil, msg) + fs.StoreMsg("foo.BB.bar", nil, msg, 0) } // Should have created 7 blocks. @@ -7582,10 +7582,10 @@ func TestFileStoreSyncCompressOnlyIfDirty(t *testing.T) { // 6 msgs per block. // Fill 2 blocks. for i := 0; i < 12; i++ { - fs.StoreMsg("foo.BB", nil, msg) + fs.StoreMsg("foo.BB", nil, msg, 0) } // Create third block with just one message in it. - fs.StoreMsg("foo.BB", nil, msg) + fs.StoreMsg("foo.BB", nil, msg, 0) // Should have created 3 blocks. require_Equal(t, fs.numMsgBlocks(), 3) @@ -7597,7 +7597,7 @@ func TestFileStoreSyncCompressOnlyIfDirty(t *testing.T) { } // Now make sure we add 4/5th block so syncBlocks will try to compact. for i := 0; i < 6; i++ { - fs.StoreMsg("foo.BB", nil, msg) + fs.StoreMsg("foo.BB", nil, msg, 0) } require_Equal(t, fs.numMsgBlocks(), 5) @@ -7662,13 +7662,13 @@ func TestFileStoreDmapBlockRecoverAfterCompact(t *testing.T) { // 6 msgs per block. // Fill the first block. for i := 1; i <= 6; i++ { - fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) } require_Equal(t, fs.numMsgBlocks(), 1) // Now create holes in the first block via the max msgs per subject of 1. for i := 2; i < 6; i++ { - fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) } require_Equal(t, fs.numMsgBlocks(), 2) // Compact and rebuild the first blk. Do not have it call indexCacheBuf which will fix it up. @@ -7700,7 +7700,7 @@ func TestFileStoreRestoreIndexWithMatchButLeftOverBlocks(t *testing.T) { // 6 msgs per block. // Fill the first 2 blocks. for i := 1; i <= 12; i++ { - fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) } require_Equal(t, fs.numMsgBlocks(), 2) @@ -7724,7 +7724,7 @@ func TestFileStoreRestoreIndexWithMatchButLeftOverBlocks(t *testing.T) { defer fs.Stop() for i := 1; i <= 6; i++ { - fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) } // Grab correct state, we will use it to make sure we do the right thing. @@ -7773,7 +7773,7 @@ func TestFileStoreRestoreDeleteTombstonesExceedingMaxBlkSize(t *testing.T) { msg := []byte("hello") // 6 msgs per block with blk size 256. for i := 1; i <= 10_000; i++ { - fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) } // Now delete msgs which will write tombstones. for seq := uint64(1_000_000_001); seq < 1_000_000_101; seq++ { @@ -7853,7 +7853,7 @@ func Benchmark_FileStoreSelectMsgBlock(b *testing.B) { // Add in a bunch of blocks. for i := 0; i < 1000; i++ { - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } if fs.numMsgBlocks() < 1000 { b.Fatalf("Expected at least 1000 blocks, got %d", fs.numMsgBlocks()) @@ -7885,7 +7885,7 @@ func Benchmark_FileStoreLoadNextMsgSameFilterAsStream(b *testing.B) { // Add in a bunch of msgs for i := 0; i < 100_000; i++ { subj := fmt.Sprintf("foo.%d", rand.Intn(1024)) - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } b.ResetTimer() @@ -7911,12 +7911,12 @@ func Benchmark_FileStoreLoadNextMsgLiteralSubject(b *testing.B) { // Add in a bunch of msgs for i := 0; i < 100_000; i++ { subj := fmt.Sprintf("foo.%d", rand.Intn(1024)) - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // This is the one we will try to match. - fs.StoreMsg("foo.2222", nil, msg) + fs.StoreMsg("foo.2222", nil, msg, 0) // So not last and we think we are done linear scan. - fs.StoreMsg("foo.3333", nil, msg) + fs.StoreMsg("foo.3333", nil, msg, 0) b.ResetTimer() @@ -7939,7 +7939,7 @@ func Benchmark_FileStoreLoadNextMsgNoMsgsFirstSeq(b *testing.B) { // Add in a bunch of msgs for i := 0; i < 1_000_000; i++ { - fs.StoreMsg("foo.bar", nil, msg) + fs.StoreMsg("foo.bar", nil, msg, 0) } b.ResetTimer() @@ -7966,7 +7966,7 @@ func Benchmark_FileStoreLoadNextMsgNoMsgsNotFirstSeq(b *testing.B) { // Add in a bunch of msgs for i := 0; i < 1_000_000; i++ { - fs.StoreMsg("foo.bar", nil, msg) + fs.StoreMsg("foo.bar", nil, msg, 0) } b.ResetTimer() @@ -7994,10 +7994,10 @@ func Benchmark_FileStoreLoadNextMsgVerySparseMsgsFirstSeq(b *testing.B) { // Add in a bunch of msgs for i := 0; i < 1_000_000; i++ { - fs.StoreMsg("foo.bar", nil, msg) + fs.StoreMsg("foo.bar", nil, msg, 0) } // Make last msg one that would match. - fs.StoreMsg("foo.baz", nil, msg) + fs.StoreMsg("foo.baz", nil, msg, 0) b.ResetTimer() @@ -8020,10 +8020,10 @@ func Benchmark_FileStoreLoadNextMsgVerySparseMsgsNotFirstSeq(b *testing.B) { // Add in a bunch of msgs for i := 0; i < 1_000_000; i++ { - fs.StoreMsg("foo.bar", nil, msg) + fs.StoreMsg("foo.bar", nil, msg, 0) } // Make last msg one that would match. - fs.StoreMsg("foo.baz", nil, msg) + fs.StoreMsg("foo.baz", nil, msg, 0) b.ResetTimer() @@ -8046,13 +8046,13 @@ func Benchmark_FileStoreLoadNextMsgVerySparseMsgsInBetween(b *testing.B) { msg := []byte("ok") // Make first msg one that would match as well. - fs.StoreMsg("foo.baz", nil, msg) + fs.StoreMsg("foo.baz", nil, msg, 0) // Add in a bunch of msgs for i := 0; i < 1_000_000; i++ { - fs.StoreMsg("foo.bar", nil, msg) + fs.StoreMsg("foo.bar", nil, msg, 0) } // Make last msg one that would match as well. - fs.StoreMsg("foo.baz", nil, msg) + fs.StoreMsg("foo.baz", nil, msg, 0) b.ResetTimer() @@ -8075,15 +8075,15 @@ func Benchmark_FileStoreLoadNextMsgVerySparseMsgsInBetweenWithWildcard(b *testin msg := []byte("ok") // Make first msg one that would match as well. - fs.StoreMsg("foo.1.baz", nil, msg) + fs.StoreMsg("foo.1.baz", nil, msg, 0) // Add in a bunch of msgs. // We need to make sure we have a range of subjects that could kick in a linear scan. for i := 0; i < 1_000_000; i++ { subj := fmt.Sprintf("foo.%d.bar", rand.Intn(100_000)+2) - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // Make last msg one that would match as well. - fs.StoreMsg("foo.1.baz", nil, msg) + fs.StoreMsg("foo.1.baz", nil, msg, 0) b.ResetTimer() @@ -8106,15 +8106,15 @@ func Benchmark_FileStoreLoadNextManySubjectsWithWildcardNearLastBlock(b *testing msg := []byte("ok") // Make first msg one that would match as well. - fs.StoreMsg("foo.1.baz", nil, msg) + fs.StoreMsg("foo.1.baz", nil, msg, 0) // Add in a bunch of msgs. // We need to make sure we have a range of subjects that could kick in a linear scan. for i := 0; i < 1_000_000; i++ { subj := fmt.Sprintf("foo.%d.bar", rand.Intn(100_000)+2) - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // Make last msg one that would match as well. - fs.StoreMsg("foo.1.baz", nil, msg) + fs.StoreMsg("foo.1.baz", nil, msg, 0) b.ResetTimer() @@ -8137,12 +8137,12 @@ func Benchmark_FileStoreLoadNextMsgVerySparseMsgsLargeTail(b *testing.B) { msg := []byte("ok") // Make first msg one that would match as well. - fs.StoreMsg("foo.1.baz", nil, msg) + fs.StoreMsg("foo.1.baz", nil, msg, 0) // Add in a bunch of msgs. // We need to make sure we have a range of subjects that could kick in a linear scan. for i := 0; i < 1_000_000; i++ { subj := fmt.Sprintf("foo.%d.bar", rand.Intn(64_000)+2) - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } b.ResetTimer() @@ -8190,7 +8190,7 @@ func TestFileStoreWriteFullStateDetectCorruptState(t *testing.T) { msg := []byte("abc") for i := 1; i <= 10; i++ { - _, _, err = fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + _, _, err = fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) require_NoError(t, err) } @@ -8225,7 +8225,7 @@ func TestFileStoreRecoverFullStateDetectCorruptState(t *testing.T) { msg := []byte("abc") for i := 1; i <= 10; i++ { - _, _, err = fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + _, _, err = fs.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) require_NoError(t, err) } @@ -8268,7 +8268,7 @@ func TestFileStoreNumPendingMulti(t *testing.T) { // Put in 100k msgs with random subjects. msg := bytes.Repeat([]byte("ZZZ"), 333) for i := 0; i < totalMsgs; i++ { - _, _, err = fs.StoreMsg(subjects[rand.Intn(totalSubjects)], nil, msg) + _, _, err = fs.StoreMsg(subjects[rand.Intn(totalSubjects)], nil, msg, 0) require_NoError(t, err) } @@ -8296,3 +8296,190 @@ func TestFileStoreNumPendingMulti(t *testing.T) { } require_Equal(t, total, checkTotal) } + +func TestFileStoreMessageTTL(t *testing.T) { + fs, err := newFileStore( + FileStoreConfig{StoreDir: t.TempDir(), EnforceTTLs: true}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + require_NoError(t, err) + defer fs.Stop() + + ttl := int64(1) // 1 second + + for i := 1; i <= 10; i++ { + _, _, err = fs.StoreMsg("test", nil, nil, ttl) + require_NoError(t, err) + } + + var ss StreamState + fs.FastState(&ss) + require_Equal(t, ss.FirstSeq, 1) + require_Equal(t, ss.LastSeq, 10) + require_Equal(t, ss.Msgs, 10) + + time.Sleep(time.Second * 2) + + fs.FastState(&ss) + require_Equal(t, ss.FirstSeq, 11) + require_Equal(t, ss.LastSeq, 10) + require_Equal(t, ss.Msgs, 0) +} + +func TestFileStoreMessageTTLRestart(t *testing.T) { + dir := t.TempDir() + + t.Run("BeforeRestart", func(t *testing.T) { + fs, err := newFileStore( + FileStoreConfig{StoreDir: dir, EnforceTTLs: true}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + require_NoError(t, err) + defer fs.Stop() + + ttl := int64(1) // 1 second + + for i := 1; i <= 10; i++ { + _, _, err = fs.StoreMsg("test", nil, nil, ttl) + require_NoError(t, err) + } + + var ss StreamState + fs.FastState(&ss) + require_Equal(t, ss.FirstSeq, 1) + require_Equal(t, ss.LastSeq, 10) + require_Equal(t, ss.Msgs, 10) + }) + + t.Run("AfterRestart", func(t *testing.T) { + fs, err := newFileStore( + FileStoreConfig{StoreDir: dir, EnforceTTLs: true}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + require_NoError(t, err) + defer fs.Stop() + + var ss StreamState + fs.FastState(&ss) + require_Equal(t, ss.FirstSeq, 1) + require_Equal(t, ss.LastSeq, 10) + require_Equal(t, ss.Msgs, 10) + + time.Sleep(time.Second * 2) + + fs.FastState(&ss) + require_Equal(t, ss.FirstSeq, 11) + require_Equal(t, ss.LastSeq, 10) + require_Equal(t, ss.Msgs, 0) + }) +} + +func TestFileStoreMessageTTLRecovered(t *testing.T) { + s := RunBasicJetStreamServer(t) + defer s.Shutdown() + + dir := t.TempDir() + + t.Run("BeforeRestart", func(t *testing.T) { + fs, err := newFileStore( + FileStoreConfig{StoreDir: dir, EnforceTTLs: true, srv: s}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + require_NoError(t, err) + defer fs.Stop() + + ttl := int64(1) // 1 second + + for i := 1; i <= 10; i++ { + // When the timed hash wheel state is deleted, the only way we can recover + // the TTL is to look at the original message header, therefore the TTL + // must be in the headers for this test to work. + hdr := fmt.Appendf(nil, "NATS/1.0\r\n%s: %d\r\n", JSMessageTTL, ttl) + _, _, err = fs.StoreMsg("test", hdr, nil, ttl) + require_NoError(t, err) + } + + var ss StreamState + fs.FastState(&ss) + require_Equal(t, ss.FirstSeq, 1) + require_Equal(t, ss.LastSeq, 10) + require_Equal(t, ss.Msgs, 10) + }) + + t.Run("AfterRestart", func(t *testing.T) { + // Delete the timed hash wheel state so that we are forced to do a linear scan + // of message blocks containing TTL'd messages. + fn := filepath.Join(dir, msgDir, ttlStreamStateFile) + require_NoError(t, os.RemoveAll(fn)) + + fs, err := newFileStore( + FileStoreConfig{StoreDir: dir, EnforceTTLs: true, srv: s}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + require_NoError(t, err) + defer fs.Stop() + + var ss StreamState + fs.FastState(&ss) + require_Equal(t, ss.FirstSeq, 1) + require_Equal(t, ss.LastSeq, 10) + require_Equal(t, ss.Msgs, 10) + + time.Sleep(time.Second * 2) + + fs.FastState(&ss) + require_Equal(t, ss.FirstSeq, 11) + require_Equal(t, ss.LastSeq, 10) + require_Equal(t, ss.Msgs, 0) + }) +} + +func TestFileStoreMessageTTLRecoveredSingleMessageWithoutStreamState(t *testing.T) { + s := RunBasicJetStreamServer(t) + defer s.Shutdown() + + dir := t.TempDir() + + t.Run("BeforeRestart", func(t *testing.T) { + fs, err := newFileStore( + FileStoreConfig{StoreDir: dir, EnforceTTLs: true, srv: s}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + require_NoError(t, err) + defer fs.Stop() + + ttl := int64(1) // 1 second + hdr := fmt.Appendf(nil, "NATS/1.0\r\n%s: %d\r\n", JSMessageTTL, ttl) + _, _, err = fs.StoreMsg("test", hdr, nil, ttl) + require_NoError(t, err) + + var ss StreamState + fs.FastState(&ss) + require_Equal(t, ss.FirstSeq, 1) + require_Equal(t, ss.LastSeq, 1) + require_Equal(t, ss.Msgs, 1) + }) + + t.Run("AfterRestart", func(t *testing.T) { + // Delete the stream state file so that we need to rebuild. + fn := filepath.Join(dir, msgDir, streamStreamStateFile) + require_NoError(t, os.RemoveAll(fn)) + // Delete the timed hash wheel state so that we are forced to do a linear scan + // of message blocks containing TTL'd messages. + fn = filepath.Join(dir, msgDir, ttlStreamStateFile) + require_NoError(t, os.RemoveAll(fn)) + + fs, err := newFileStore( + FileStoreConfig{StoreDir: dir, EnforceTTLs: true, srv: s}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + require_NoError(t, err) + defer fs.Stop() + + var ss StreamState + fs.FastState(&ss) + require_Equal(t, ss.FirstSeq, 1) + require_Equal(t, ss.LastSeq, 1) + require_Equal(t, ss.Msgs, 1) + + time.Sleep(time.Second * 2) + + fs.FastState(&ss) + require_Equal(t, ss.FirstSeq, 2) + require_Equal(t, ss.LastSeq, 1) + require_Equal(t, ss.Msgs, 0) + }) +} diff --git a/server/jetstream_cluster.go b/server/jetstream_cluster.go index c54ab05cf4b..e5ef00eae11 100644 --- a/server/jetstream_cluster.go +++ b/server/jetstream_cluster.go @@ -8826,7 +8826,7 @@ func (mset *stream) processCatchupMsg(msg []byte) (uint64, error) { if lseq := mset.store.SkipMsg(); lseq != seq { return 0, errCatchupWrongSeqForSkip } - } else if err := mset.store.StoreRawMsg(subj, hdr, msg, seq, ts); err != nil { + } else if err := mset.store.StoreRawMsg(subj, hdr, msg, seq, ts, 0); err != nil { return 0, err } diff --git a/server/jetstream_errors_generated.go b/server/jetstream_errors_generated.go index 146e969eb7c..c954462dc5d 100644 --- a/server/jetstream_errors_generated.go +++ b/server/jetstream_errors_generated.go @@ -242,6 +242,9 @@ const ( // JSMemoryResourcesExceededErr insufficient memory resources available JSMemoryResourcesExceededErr ErrorIdentifier = 10028 + // JSMessageTTLInvalidErr invalid per-message TTL + JSMessageTTLInvalidErr ErrorIdentifier = 10165 + // JSMirrorConsumerSetupFailedErrF generic mirror consumer setup failure string ({err}) JSMirrorConsumerSetupFailedErrF ErrorIdentifier = 10029 @@ -576,6 +579,7 @@ var ( JSMaximumConsumersLimitErr: {Code: 400, ErrCode: 10026, Description: "maximum consumers limit reached"}, JSMaximumStreamsLimitErr: {Code: 400, ErrCode: 10027, Description: "maximum number of streams reached"}, JSMemoryResourcesExceededErr: {Code: 500, ErrCode: 10028, Description: "insufficient memory resources available"}, + JSMessageTTLInvalidErr: {Code: 400, ErrCode: 10165, Description: "invalid per-message TTL"}, JSMirrorConsumerSetupFailedErrF: {Code: 500, ErrCode: 10029, Description: "{err}"}, JSMirrorInvalidStreamName: {Code: 400, ErrCode: 10142, Description: "mirrored stream name is invalid"}, JSMirrorInvalidSubjectFilter: {Code: 400, ErrCode: 10151, Description: "mirror transform source: {err}"}, @@ -1547,6 +1551,16 @@ func NewJSMemoryResourcesExceededError(opts ...ErrorOption) *ApiError { return ApiErrors[JSMemoryResourcesExceededErr] } +// NewJSMessageTTLInvalidError creates a new JSMessageTTLInvalidErr error: "invalid per-message TTL" +func NewJSMessageTTLInvalidError(opts ...ErrorOption) *ApiError { + eopts := parseOpts(opts) + if ae, ok := eopts.err.(*ApiError); ok { + return ae + } + + return ApiErrors[JSMessageTTLInvalidErr] +} + // NewJSMirrorConsumerSetupFailedError creates a new JSMirrorConsumerSetupFailedErrF error: "{err}" func NewJSMirrorConsumerSetupFailedError(err error, opts ...ErrorOption) *ApiError { eopts := parseOpts(opts) diff --git a/server/jetstream_helpers_test.go b/server/jetstream_helpers_test.go index fccb2646da1..4afbd952d3a 100644 --- a/server/jetstream_helpers_test.go +++ b/server/jetstream_helpers_test.go @@ -1235,6 +1235,39 @@ func jsClientConnectURL(t testing.TB, url string, opts ...nats.Option) (*nats.Co return nc, js } +// jsStreamCreate is for sending a stream create for fields that nats.go does not know about yet. +func jsStreamCreate(t testing.TB, nc *nats.Conn, cfg *StreamConfig) *StreamConfig { + j, err := json.Marshal(cfg) + require_NoError(t, err) + + msg, err := nc.Request(fmt.Sprintf(JSApiStreamCreateT, cfg.Name), j, time.Second*3) + require_NoError(t, err) + + var resp JSApiStreamUpdateResponse + require_NoError(t, json.Unmarshal(msg.Data, &resp)) + require_NotNil(t, resp.StreamInfo) + return &resp.Config +} + +// jsStreamUpdate is for sending a stream create for fields that nats.go does not know about yet. +func jsStreamUpdate(t testing.TB, nc *nats.Conn, cfg *StreamConfig) (*StreamConfig, error) { + j, err := json.Marshal(cfg) + require_NoError(t, err) + + msg, err := nc.Request(fmt.Sprintf(JSApiStreamUpdateT, cfg.Name), j, time.Second*3) + require_NoError(t, err) + + var resp JSApiStreamUpdateResponse + require_NoError(t, json.Unmarshal(msg.Data, &resp)) + + if resp.Error != nil { + return nil, resp.Error + } + + require_NotNil(t, resp.StreamInfo) + return &resp.Config, nil +} + func checkSubsPending(t *testing.T, sub *nats.Subscription, numExpected int) { t.Helper() checkFor(t, 10*time.Second, 20*time.Millisecond, func() error { diff --git a/server/jetstream_test.go b/server/jetstream_test.go index f578b123071..305091bbdee 100644 --- a/server/jetstream_test.go +++ b/server/jetstream_test.go @@ -24802,3 +24802,207 @@ func TestJetStreamConsumerDecrementPendingCountOnSkippedMsg(t *testing.T) { o.decStreamPending(3, "foo") requireExpected(0) } + +func TestJetStreamMessageTTL(t *testing.T) { + s := RunBasicJetStreamServer(t) + defer s.Shutdown() + + nc, js := jsClientConnect(t, s) + defer nc.Close() + + jsStreamCreate(t, nc, &StreamConfig{ + Name: "TEST", + Storage: FileStorage, + Subjects: []string{"test"}, + AllowMsgTTL: true, + }) + + msg := &nats.Msg{ + Subject: "test", + Header: nats.Header{}, + } + + for i := 1; i <= 10; i++ { + msg.Header.Set("Nats-TTL", "1s") + _, err := js.PublishMsg(msg) + require_NoError(t, err) + } + + si, err := js.StreamInfo("TEST") + require_NoError(t, err) + require_Equal(t, si.State.Msgs, 10) + require_Equal(t, si.State.FirstSeq, 1) + require_Equal(t, si.State.LastSeq, 10) + + time.Sleep(time.Second * 2) + + si, err = js.StreamInfo("TEST") + require_NoError(t, err) + require_Equal(t, si.State.Msgs, 0) + require_Equal(t, si.State.FirstSeq, 11) + require_Equal(t, si.State.LastSeq, 10) +} + +func TestJetStreamMessageTTLRestart(t *testing.T) { + s := RunBasicJetStreamServer(t) + defer s.Shutdown() + + nc, js := jsClientConnect(t, s) + defer nc.Close() + + jsStreamCreate(t, nc, &StreamConfig{ + Name: "TEST", + Storage: FileStorage, + Subjects: []string{"test"}, + AllowMsgTTL: true, + }) + + msg := &nats.Msg{ + Subject: "test", + Header: nats.Header{}, + } + + for i := 1; i <= 10; i++ { + msg.Header.Set("Nats-TTL", "1s") + _, err := js.PublishMsg(msg) + require_NoError(t, err) + } + + si, err := js.StreamInfo("TEST") + require_NoError(t, err) + require_Equal(t, si.State.Msgs, 10) + require_Equal(t, si.State.FirstSeq, 1) + require_Equal(t, si.State.LastSeq, 10) + + sd := s.JetStreamConfig().StoreDir + s.Shutdown() + + s = RunJetStreamServerOnPort(-1, sd) + defer s.Shutdown() + + nc, js = jsClientConnect(t, s) + defer nc.Close() + + si, err = js.StreamInfo("TEST") + require_NoError(t, err) + require_Equal(t, si.State.Msgs, 10) + require_Equal(t, si.State.FirstSeq, 1) + require_Equal(t, si.State.LastSeq, 10) + + time.Sleep(time.Second * 2) + + si, err = js.StreamInfo("TEST") + require_NoError(t, err) + require_Equal(t, si.State.Msgs, 0) + require_Equal(t, si.State.FirstSeq, 11) + require_Equal(t, si.State.LastSeq, 10) +} + +func TestJetStreamMessageTTLRecovered(t *testing.T) { + s := RunBasicJetStreamServer(t) + defer s.Shutdown() + + nc, js := jsClientConnect(t, s) + defer nc.Close() + + jsStreamCreate(t, nc, &StreamConfig{ + Name: "TEST", + Storage: FileStorage, + Subjects: []string{"test"}, + AllowMsgTTL: true, + }) + + msg := &nats.Msg{ + Subject: "test", + Header: nats.Header{}, + } + + for i := 1; i <= 10; i++ { + msg.Header.Set("Nats-TTL", "1s") + _, err := js.PublishMsg(msg) + require_NoError(t, err) + } + + si, err := js.StreamInfo("TEST") + require_NoError(t, err) + require_Equal(t, si.State.Msgs, 10) + require_Equal(t, si.State.FirstSeq, 1) + require_Equal(t, si.State.LastSeq, 10) + + sd := s.JetStreamConfig().StoreDir + s.Shutdown() + + fn := filepath.Join(sd, globalAccountName, streamsDir, "TEST", msgDir, ttlStreamStateFile) + require_NoError(t, os.RemoveAll(fn)) + + s = RunJetStreamServerOnPort(-1, sd) + defer s.Shutdown() + + nc, js = jsClientConnect(t, s) + defer nc.Close() + + si, err = js.StreamInfo("TEST") + require_NoError(t, err) + require_Equal(t, si.State.Msgs, 10) + require_Equal(t, si.State.FirstSeq, 1) + require_Equal(t, si.State.LastSeq, 10) + + time.Sleep(time.Second * 2) + + si, err = js.StreamInfo("TEST") + require_NoError(t, err) + require_Equal(t, si.State.Msgs, 0) + require_Equal(t, si.State.FirstSeq, 11) + require_Equal(t, si.State.LastSeq, 10) +} + +func TestJetStreamMessageTTLInvalid(t *testing.T) { + s := RunBasicJetStreamServer(t) + defer s.Shutdown() + + nc, js := jsClientConnect(t, s) + defer nc.Close() + + jsStreamCreate(t, nc, &StreamConfig{ + Name: "TEST", + Storage: FileStorage, + Subjects: []string{"test"}, + AllowMsgTTL: true, + }) + + msg := &nats.Msg{ + Subject: "test", + Header: nats.Header{}, + } + + msg.Header.Set("Nats-TTL", "500ms") + _, err := js.PublishMsg(msg) + require_Error(t, err) + + msg.Header.Set("Nats-TTL", "something") + _, err = js.PublishMsg(msg) + require_Error(t, err) +} + +func TestJetStreamMessageTTLNotUpdatable(t *testing.T) { + s := RunBasicJetStreamServer(t) + defer s.Shutdown() + + nc, _ := jsClientConnect(t, s) + defer nc.Close() + + jsStreamCreate(t, nc, &StreamConfig{ + Name: "TEST", + Storage: FileStorage, + Subjects: []string{"test"}, + AllowMsgTTL: true, + }) + + _, err := jsStreamUpdate(t, nc, &StreamConfig{ + Name: "TEST", + Storage: FileStorage, + Subjects: []string{"test"}, + AllowMsgTTL: false, + }) + require_Error(t, err) +} diff --git a/server/memstore.go b/server/memstore.go index 2c867036f87..5f76fed8fd5 100644 --- a/server/memstore.go +++ b/server/memstore.go @@ -110,7 +110,7 @@ func (ms *memStore) UpdateConfig(cfg *StreamConfig) error { // Stores a raw message with expected sequence number and timestamp. // Lock should be held. -func (ms *memStore) storeRawMsg(subj string, hdr, msg []byte, seq uint64, ts int64) error { +func (ms *memStore) storeRawMsg(subj string, hdr, msg []byte, seq uint64, ts, ttl int64) error { if ms.msgs == nil { return ErrStoreClosed } @@ -218,9 +218,9 @@ func (ms *memStore) storeRawMsg(subj string, hdr, msg []byte, seq uint64, ts int } // StoreRawMsg stores a raw message with expected sequence number and timestamp. -func (ms *memStore) StoreRawMsg(subj string, hdr, msg []byte, seq uint64, ts int64) error { +func (ms *memStore) StoreRawMsg(subj string, hdr, msg []byte, seq uint64, ts, ttl int64) error { ms.mu.Lock() - err := ms.storeRawMsg(subj, hdr, msg, seq, ts) + err := ms.storeRawMsg(subj, hdr, msg, seq, ts, ttl) cb := ms.scb // Check if first message timestamp requires expiry // sooner than initial replica expiry timer set to MaxAge when initializing. @@ -239,10 +239,10 @@ func (ms *memStore) StoreRawMsg(subj string, hdr, msg []byte, seq uint64, ts int } // Store stores a message. -func (ms *memStore) StoreMsg(subj string, hdr, msg []byte) (uint64, int64, error) { +func (ms *memStore) StoreMsg(subj string, hdr, msg []byte, ttl int64) (uint64, int64, error) { ms.mu.Lock() seq, ts := ms.state.LastSeq+1, time.Now().UnixNano() - err := ms.storeRawMsg(subj, hdr, msg, seq, ts) + err := ms.storeRawMsg(subj, hdr, msg, seq, ts, ttl) cb := ms.scb ms.mu.Unlock() diff --git a/server/memstore_test.go b/server/memstore_test.go index cfc2aa95bd5..5458a31b806 100644 --- a/server/memstore_test.go +++ b/server/memstore_test.go @@ -35,7 +35,7 @@ func TestMemStoreBasics(t *testing.T) { subj, msg := "foo", []byte("Hello World") now := time.Now().UnixNano() - if seq, ts, err := ms.StoreMsg(subj, nil, msg); err != nil { + if seq, ts, err := ms.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Error storing msg: %v", err) } else if seq != 1 { t.Fatalf("Expected sequence to be 1, got %d", seq) @@ -70,13 +70,13 @@ func TestMemStoreMsgLimit(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 0; i < 10; i++ { - ms.StoreMsg(subj, nil, msg) + ms.StoreMsg(subj, nil, msg, 0) } state := ms.State() if state.Msgs != 10 { t.Fatalf("Expected %d msgs, got %d", 10, state.Msgs) } - if _, _, err := ms.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := ms.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Error storing msg: %v", err) } state = ms.State() @@ -107,7 +107,7 @@ func TestMemStoreBytesLimit(t *testing.T) { defer ms.Stop() for i := uint64(0); i < toStore; i++ { - ms.StoreMsg(subj, nil, msg) + ms.StoreMsg(subj, nil, msg, 0) } state := ms.State() if state.Msgs != toStore { @@ -119,7 +119,7 @@ func TestMemStoreBytesLimit(t *testing.T) { // Now send 10 more and check that bytes limit enforced. for i := 0; i < 10; i++ { - if _, _, err := ms.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := ms.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Error storing msg: %v", err) } } @@ -152,7 +152,7 @@ func TestMemStoreBytesLimitWithDiscardNew(t *testing.T) { // Now send 10 messages and check that bytes limit enforced. for i := 0; i < 10; i++ { - _, _, err := ms.StoreMsg(subj, nil, msg) + _, _, err := ms.StoreMsg(subj, nil, msg, 0) if i < int(toStore) { if err != nil { t.Fatalf("Error storing msg: %v", err) @@ -180,7 +180,7 @@ func TestMemStoreAgeLimit(t *testing.T) { subj, msg := "foo", []byte("Hello World") toStore := 100 for i := 0; i < toStore; i++ { - ms.StoreMsg(subj, nil, msg) + ms.StoreMsg(subj, nil, msg, 0) } state := ms.State() if state.Msgs != uint64(toStore) { @@ -203,7 +203,7 @@ func TestMemStoreAgeLimit(t *testing.T) { checkExpired(t) // Now add some more and make sure that timer will fire again. for i := 0; i < toStore; i++ { - ms.StoreMsg(subj, nil, msg) + ms.StoreMsg(subj, nil, msg, 0) } state = ms.State() if state.Msgs != uint64(toStore) { @@ -221,7 +221,7 @@ func TestMemStoreTimeStamps(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 0; i < 10; i++ { time.Sleep(5 * time.Microsecond) - ms.StoreMsg(subj, nil, msg) + ms.StoreMsg(subj, nil, msg, 0) } var smv StoreMsg for seq := uint64(1); seq <= 10; seq++ { @@ -244,7 +244,7 @@ func TestMemStorePurge(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 0; i < 10; i++ { - ms.StoreMsg(subj, nil, msg) + ms.StoreMsg(subj, nil, msg, 0) } if state := ms.State(); state.Msgs != 10 { t.Fatalf("Expected 10 msgs, got %d", state.Msgs) @@ -262,7 +262,7 @@ func TestMemStoreCompact(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 0; i < 10; i++ { - ms.StoreMsg(subj, nil, msg) + ms.StoreMsg(subj, nil, msg, 0) } if state := ms.State(); state.Msgs != 10 { t.Fatalf("Expected 10 msgs, got %d", state.Msgs) @@ -300,7 +300,7 @@ func TestMemStoreEraseMsg(t *testing.T) { defer ms.Stop() subj, msg := "foo", []byte("Hello World") - ms.StoreMsg(subj, nil, msg) + ms.StoreMsg(subj, nil, msg, 0) sm, err := ms.LoadMsg(1, nil) if err != nil { t.Fatalf("Unexpected error looking up msg: %v", err) @@ -322,7 +322,7 @@ func TestMemStoreMsgHeaders(t *testing.T) { if sz := int(memStoreMsgSize(subj, hdr, msg)); sz != (len(subj) + len(hdr) + len(msg) + 16) { t.Fatalf("Wrong size for stored msg with header") } - ms.StoreMsg(subj, hdr, msg) + ms.StoreMsg(subj, hdr, msg, 0) sm, err := ms.LoadMsg(1, nil) if err != nil { t.Fatalf("Unexpected error looking up msg: %v", err) @@ -346,7 +346,7 @@ func TestMemStoreStreamStateDeleted(t *testing.T) { subj, toStore := "foo", uint64(10) for i := uint64(1); i <= toStore; i++ { msg := []byte(fmt.Sprintf("[%08d] Hello World!", i)) - if _, _, err := ms.StoreMsg(subj, nil, msg); err != nil { + if _, _, err := ms.StoreMsg(subj, nil, msg, 0); err != nil { t.Fatalf("Error storing msg: %v", err) } } @@ -390,12 +390,12 @@ func TestMemStoreStreamTruncate(t *testing.T) { subj, toStore := "foo", uint64(100) for i := uint64(1); i < tseq; i++ { - _, _, err := ms.StoreMsg(subj, nil, []byte("ok")) + _, _, err := ms.StoreMsg(subj, nil, []byte("ok"), 0) require_NoError(t, err) } subj = "bar" for i := tseq; i <= toStore; i++ { - _, _, err := ms.StoreMsg(subj, nil, []byte("ok")) + _, _, err := ms.StoreMsg(subj, nil, []byte("ok"), 0) require_NoError(t, err) } @@ -444,7 +444,7 @@ func TestMemStorePurgeExWithSubject(t *testing.T) { defer ms.Stop() for i := 0; i < 100; i++ { - _, _, err = ms.StoreMsg("foo", nil, nil) + _, _, err = ms.StoreMsg("foo", nil, nil, 0) require_NoError(t, err) } @@ -471,7 +471,7 @@ func TestMemStoreUpdateMaxMsgsPerSubject(t *testing.T) { numStored := 22 for i := 0; i < numStored; i++ { - _, _, err = ms.StoreMsg("foo", nil, nil) + _, _, err = ms.StoreMsg("foo", nil, nil, 0) require_NoError(t, err) } @@ -503,7 +503,7 @@ func TestMemStoreStreamTruncateReset(t *testing.T) { subj, msg := "foo", []byte("Hello World") for i := 0; i < 1000; i++ { - _, _, err := ms.StoreMsg(subj, nil, msg) + _, _, err := ms.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } @@ -519,7 +519,7 @@ func TestMemStoreStreamTruncateReset(t *testing.T) { require_True(t, state.NumDeleted == 0) for i := 0; i < 1000; i++ { - _, _, err := ms.StoreMsg(subj, nil, msg) + _, _, err := ms.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } @@ -544,7 +544,7 @@ func TestMemStoreStreamCompactMultiBlockSubjectInfo(t *testing.T) { for i := 0; i < 1000; i++ { subj := fmt.Sprintf("foo.%d", i) - _, _, err := ms.StoreMsg(subj, nil, []byte("Hello World")) + _, _, err := ms.StoreMsg(subj, nil, []byte("Hello World"), 0) require_NoError(t, err) } @@ -585,7 +585,7 @@ func TestMemStoreSubjectsTotals(t *testing.T) { subj := fmt.Sprintf("%s.%d", ft, dt) m[dt]++ - _, _, err := ms.StoreMsg(subj, nil, []byte("Hello World")) + _, _, err := ms.StoreMsg(subj, nil, []byte("Hello World"), 0) require_NoError(t, err) } @@ -660,7 +660,7 @@ func TestMemStoreNumPending(t *testing.T) { for i := 0; i < 50_000; i++ { subj := genSubj() - _, _, err := ms.StoreMsg(subj, nil, []byte("Hello World")) + _, _, err := ms.StoreMsg(subj, nil, []byte("Hello World"), 0) require_NoError(t, err) } @@ -770,13 +770,13 @@ func TestMemStoreInitialFirstSeq(t *testing.T) { require_NoError(t, err) defer ms.Stop() - seq, _, err := ms.StoreMsg("A", nil, []byte("OK")) + seq, _, err := ms.StoreMsg("A", nil, []byte("OK"), 0) require_NoError(t, err) if seq != 1000 { t.Fatalf("Message should have been sequence 1000 but was %d", seq) } - seq, _, err = ms.StoreMsg("B", nil, []byte("OK")) + seq, _, err = ms.StoreMsg("B", nil, []byte("OK"), 0) require_NoError(t, err) if seq != 1001 { t.Fatalf("Message should have been sequence 1001 but was %d", seq) @@ -807,7 +807,7 @@ func TestMemStoreDeleteBlocks(t *testing.T) { // Put in 10_000 msgs. total := 10_000 for i := 0; i < total; i++ { - _, _, err := ms.StoreMsg("A", nil, []byte("OK")) + _, _, err := ms.StoreMsg("A", nil, []byte("OK"), 0) require_NoError(t, err) } @@ -848,7 +848,7 @@ func TestMemStoreGetSeqFromTimeWithLastDeleted(t *testing.T) { total := 1000 var st time.Time for i := 1; i <= total; i++ { - _, _, err := ms.StoreMsg("A", nil, []byte("OK")) + _, _, err := ms.StoreMsg("A", nil, []byte("OK"), 0) require_NoError(t, err) if i == total/2 { time.Sleep(100 * time.Millisecond) @@ -898,7 +898,7 @@ func TestMemStoreSkipMsgs(t *testing.T) { // Now add in a message, and then skip to check dmap. ms, err = newMemStore(cfg) require_NoError(t, err) - ms.StoreMsg("foo", nil, nil) + ms.StoreMsg("foo", nil, nil, 0) err = ms.SkipMsgs(2, 10) require_NoError(t, err) @@ -930,14 +930,14 @@ func TestMemStoreMultiLastSeqs(t *testing.T) { msg := []byte("abc") for i := 0; i < 33; i++ { - ms.StoreMsg("foo.foo", nil, msg) - ms.StoreMsg("foo.bar", nil, msg) - ms.StoreMsg("foo.baz", nil, msg) + ms.StoreMsg("foo.foo", nil, msg, 0) + ms.StoreMsg("foo.bar", nil, msg, 0) + ms.StoreMsg("foo.baz", nil, msg, 0) } for i := 0; i < 33; i++ { - ms.StoreMsg("bar.foo", nil, msg) - ms.StoreMsg("bar.bar", nil, msg) - ms.StoreMsg("bar.baz", nil, msg) + ms.StoreMsg("bar.foo", nil, msg, 0) + ms.StoreMsg("bar.bar", nil, msg, 0) + ms.StoreMsg("bar.baz", nil, msg, 0) } checkResults := func(seqs, expected []uint64) { @@ -1017,7 +1017,7 @@ func TestMemStoreMultiLastSeqsMaxAllowed(t *testing.T) { msg := []byte("abc") for i := 1; i <= 100; i++ { - ms.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg) + ms.StoreMsg(fmt.Sprintf("foo.%d", i), nil, msg, 0) } // Test that if we specify maxAllowed that we get the correct error. seqs, err := ms.MultiLastSeqs([]string{"foo.*"}, 0, 10) @@ -1038,7 +1038,7 @@ func TestMemStorePurgeExWithDeletedMsgs(t *testing.T) { msg := []byte("abc") for i := 1; i <= 10; i++ { - ms.StoreMsg("foo", nil, msg) + ms.StoreMsg("foo", nil, msg, 0) } ms.RemoveMsg(2) ms.RemoveMsg(9) // This was the bug @@ -1067,7 +1067,7 @@ func TestMemStoreDeleteAllFirstSequenceCheck(t *testing.T) { msg := []byte("abc") for i := 1; i <= 10; i++ { - ms.StoreMsg("foo", nil, msg) + ms.StoreMsg("foo", nil, msg, 0) } for seq := uint64(1); seq <= 10; seq++ { ms.RemoveMsg(seq) @@ -1102,7 +1102,7 @@ func TestMemStoreNumPendingMulti(t *testing.T) { // Put in 100k msgs with random subjects. msg := bytes.Repeat([]byte("ZZZ"), 333) for i := 0; i < totalMsgs; i++ { - _, _, err = ms.StoreMsg(subjects[rand.Intn(totalSubjects)], nil, msg) + _, _, err = ms.StoreMsg(subjects[rand.Intn(totalSubjects)], nil, msg, 0) require_NoError(t, err) } @@ -1143,9 +1143,9 @@ func TestMemStoreNumPendingBug(t *testing.T) { // 12 msgs total for _, subj := range []string{"foo.foo", "foo.bar", "foo.baz", "foo.zzz"} { - ms.StoreMsg("foo.aaa", nil, nil) - ms.StoreMsg(subj, nil, nil) - ms.StoreMsg(subj, nil, nil) + ms.StoreMsg("foo.aaa", nil, nil, 0) + ms.StoreMsg(subj, nil, nil, 0) + ms.StoreMsg(subj, nil, nil, 0) } total, _ := ms.NumPending(4, "foo.*", false) @@ -1176,11 +1176,11 @@ func Benchmark_MemStoreNumPendingWithLargeInteriorDeletesScan(b *testing.B) { defer ms.Stop() msg := []byte("abc") - ms.StoreMsg("foo.bar.baz", nil, msg) + ms.StoreMsg("foo.bar.baz", nil, msg, 0) for i := 1; i <= 1_000_000; i++ { ms.SkipMsg() } - ms.StoreMsg("foo.bar.baz", nil, msg) + ms.StoreMsg("foo.bar.baz", nil, msg, 0) b.ResetTimer() @@ -1203,11 +1203,11 @@ func Benchmark_MemStoreNumPendingWithLargeInteriorDeletesExclude(b *testing.B) { defer ms.Stop() msg := []byte("abc") - ms.StoreMsg("foo.bar.baz", nil, msg) + ms.StoreMsg("foo.bar.baz", nil, msg, 0) for i := 1; i <= 1_000_000; i++ { ms.SkipMsg() } - ms.StoreMsg("foo.bar.baz", nil, msg) + ms.StoreMsg("foo.bar.baz", nil, msg, 0) b.ResetTimer() diff --git a/server/norace_test.go b/server/norace_test.go index eb534987cff..69914b9dc0f 100644 --- a/server/norace_test.go +++ b/server/norace_test.go @@ -3625,7 +3625,7 @@ func TestNoRaceJetStreamClusterCorruptWAL(t *testing.T) { if err != nil { t.Fatalf("Unexpected error: %v", err) } - if _, _, err := fs.StoreMsg(_EMPTY_, nil, encoded); err != nil { + if _, _, err := fs.StoreMsg(_EMPTY_, nil, encoded, 0); err != nil { t.Fatalf("Unexpected error: %v", err) } fs.Stop() @@ -4487,7 +4487,7 @@ func TestNoRaceJetStreamFileStoreKeyFileCleanup(t *testing.T) { n, msg := 10_000, []byte(strings.Repeat("Z", 1024)) for i := 0; i < n; i++ { - _, _, err := fs.StoreMsg(fmt.Sprintf("X.%d", i), nil, msg) + _, _, err := fs.StoreMsg(fmt.Sprintf("X.%d", i), nil, msg, 0) require_NoError(t, err) } @@ -5538,7 +5538,7 @@ func TestNoRaceJetStreamFileStoreLargeKVAccessTiming(t *testing.T) { for i := 1; i <= nkeys; i++ { subj := fmt.Sprintf(tmpl, i) - _, _, err := fs.StoreMsg(subj, nil, val) + _, _, err := fs.StoreMsg(subj, nil, val, 0) require_NoError(t, err) } @@ -6270,7 +6270,7 @@ func TestNoRaceFileStoreStreamMaxAgePerformance(t *testing.T) { for time.Now().Before(timeout) { // We will store in blocks of 100. for i := 0; i < 100; i++ { - _, _, err := fs.StoreMsg(subj, nil, []byte("Hello World")) + _, _, err := fs.StoreMsg(subj, nil, []byte("Hello World"), 0) require_NoError(t, err) num++ } @@ -6288,7 +6288,7 @@ func TestNoRaceFileStoreStreamMaxAgePerformance(t *testing.T) { for time.Now().Before(timeout) { // We will store in blocks of 100. for i := 0; i < 100; i++ { - _, _, err := fs.StoreMsg(subj, nil, []byte("Hello World")) + _, _, err := fs.StoreMsg(subj, nil, []byte("Hello World"), 0) require_NoError(t, err) num++ } @@ -6362,7 +6362,7 @@ func TestNoRaceFileStoreFilteredStateWithLargeDeletes(t *testing.T) { toStore := 500_000 for i := 0; i < toStore; i++ { - _, _, err := fs.StoreMsg(subj, nil, msg) + _, _, err := fs.StoreMsg(subj, nil, msg, 0) require_NoError(t, err) } @@ -7558,7 +7558,7 @@ func TestNoRaceFileStoreNumPending(t *testing.T) { for i := 0; i < 50_000; i++ { subj := genSubj() - _, _, err := fs.StoreMsg(subj, nil, []byte("Hello World")) + _, _, err := fs.StoreMsg(subj, nil, []byte("Hello World"), 0) require_NoError(t, err) } @@ -8878,13 +8878,13 @@ func TestNoRaceFilestoreBinaryStreamSnapshotEncodingLargeGaps(t *testing.T) { subj, msg := "zzz", bytes.Repeat([]byte("X"), 128) numMsgs := 20_000 - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) for i := 2; i < numMsgs; i++ { - seq, _, err := fs.StoreMsg(subj, nil, nil) + seq, _, err := fs.StoreMsg(subj, nil, nil, 0) require_NoError(t, err) fs.RemoveMsg(seq) } - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) snap, err := fs.EncodedStreamState(0) require_NoError(t, err) @@ -9046,7 +9046,7 @@ func TestNoRaceStoreStreamEncoderDecoder(t *testing.T) { running = false default: key := strconv.Itoa(prand.Intn(256_000)) - gs.StoreMsg(key, nil, msg) + gs.StoreMsg(key, nil, msg, 0) } } } @@ -9218,10 +9218,10 @@ func TestNoRaceFileStoreLargeMsgsAndFirstMatching(t *testing.T) { defer fs.Stop() for i := 0; i < 150_000; i++ { - fs.StoreMsg(fmt.Sprintf("foo.bar.%d", i), nil, nil) + fs.StoreMsg(fmt.Sprintf("foo.bar.%d", i), nil, nil, 0) } for i := 0; i < 150_000; i++ { - fs.StoreMsg(fmt.Sprintf("foo.baz.%d", i), nil, nil) + fs.StoreMsg(fmt.Sprintf("foo.baz.%d", i), nil, nil, 0) } require_Equal(t, fs.numMsgBlocks(), 2) fs.mu.RLock() @@ -9730,7 +9730,7 @@ func TestNoRaceMemStoreCompactPerformance(t *testing.T) { defer ms.Stop() for i := uint64(0); i < toStore; i++ { - ms.StoreMsg(subj, nil, msg) + ms.StoreMsg(subj, nil, msg, 0) } state := ms.State() require_Equal(t, toStore, state.Msgs) @@ -9738,7 +9738,7 @@ func TestNoRaceMemStoreCompactPerformance(t *testing.T) { //1st run: Load additional messages then compact for i := uint64(0); i < toStoreOnTop; i++ { - ms.StoreMsg(subj, nil, msg) + ms.StoreMsg(subj, nil, msg, 0) } startFirstRun := time.Now() purgedFirstRun, _ := ms.Compact(toStore + toStoreOnTop) @@ -9751,7 +9751,7 @@ func TestNoRaceMemStoreCompactPerformance(t *testing.T) { //2nd run: Compact again for i := uint64(0); i < toStore; i++ { - ms.StoreMsg(subj, nil, msg) + ms.StoreMsg(subj, nil, msg, 0) } startSecondRun := time.Now() purgedSecondRun, _ := ms.Compact(setSeqNo + toStore - 1) @@ -10213,7 +10213,7 @@ func TestNoRaceFileStoreMsgLoadNextMsgMultiPerf(t *testing.T) { // Put 1k msgs in for i := 0; i < 1000; i++ { subj := fmt.Sprintf("foo.%d", i) - fs.StoreMsg(subj, nil, []byte("ZZZ")) + fs.StoreMsg(subj, nil, []byte("ZZZ"), 0) } var smv StoreMsg @@ -10475,7 +10475,7 @@ func TestNoRaceFileStoreWriteFullStateUniqueSubjects(t *testing.T) { partD := labels[rand.Intn(len(labels)-1)] subject := fmt.Sprintf("records.%s.%s.%s.%s.%s", partA, partB, partC, partD, nuid.Next()) start := time.Now() - fs.StoreMsg(subject, nil, msg) + fs.StoreMsg(subject, nil, msg, 0) elapsed := time.Since(start) if elapsed > 500*time.Millisecond { t.Fatalf("Slow store for %q: %v\n", subject, elapsed) @@ -10610,9 +10610,9 @@ func TestNoRaceLargeNumDeletesStreamCatchups(t *testing.T) { mset, err := sl.GlobalAccount().lookupStream("TEST") require_NoError(t, err) mset.mu.Lock() - mset.store.StoreMsg("foo", nil, []byte("ok")) + mset.store.StoreMsg("foo", nil, []byte("ok"), 0) mset.store.SkipMsgs(2, 1_000_000_000) - mset.store.StoreMsg("foo", nil, []byte("ok")) + mset.store.StoreMsg("foo", nil, []byte("ok"), 0) mset.store.SkipMsgs(1_000_000_003, 1_000_000_000) var state StreamState mset.store.FastState(&state) @@ -10985,7 +10985,7 @@ func TestNoRaceFileStoreMsgLimitsAndOldRecoverState(t *testing.T) { for i := 0; i < 10_000; i++ { subj := fmt.Sprintf("foo.%d", i) - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } // This will write the index.db file. We will capture this and use it to replace a new one. @@ -11010,7 +11010,7 @@ func TestNoRaceFileStoreMsgLimitsAndOldRecoverState(t *testing.T) { // Put in more messages with wider range. This will compact a bunch of the previous blocks. for i := 0; i < 1_000_001; i++ { subj := fmt.Sprintf("foo.%d", i) - fs.StoreMsg(subj, nil, msg) + fs.StoreMsg(subj, nil, msg, 0) } var ss StreamState diff --git a/server/raft.go b/server/raft.go index 519e6732583..72a66f3a21b 100644 --- a/server/raft.go +++ b/server/raft.go @@ -82,7 +82,7 @@ type RaftNode interface { type WAL interface { Type() StorageType - StoreMsg(subj string, hdr, msg []byte) (uint64, int64, error) + StoreMsg(subj string, hdr, msg []byte, ttl int64) (uint64, int64, error) LoadMsg(index uint64, sm *StoreMsg) (*StoreMsg, error) RemoveMsg(index uint64) (bool, error) Compact(index uint64) (uint64, error) @@ -3689,7 +3689,7 @@ func (n *raft) storeToWAL(ae *appendEntry) error { return n.werr } - seq, _, err := n.wal.StoreMsg(_EMPTY_, nil, ae.buf) + seq, _, err := n.wal.StoreMsg(_EMPTY_, nil, ae.buf, 0) if err != nil { n.setWriteErrLocked(err) return err diff --git a/server/store.go b/server/store.go index 3c24215e923..2e148dc0731 100644 --- a/server/store.go +++ b/server/store.go @@ -84,8 +84,8 @@ type StoreMsg struct { type StorageUpdateHandler func(msgs, bytes int64, seq uint64, subj string) type StreamStore interface { - StoreMsg(subject string, hdr, msg []byte) (uint64, int64, error) - StoreRawMsg(subject string, hdr, msg []byte, seq uint64, ts int64) error + StoreMsg(subject string, hdr, msg []byte, ttl int64) (uint64, int64, error) + StoreRawMsg(subject string, hdr, msg []byte, seq uint64, ts int64, ttl int64) error SkipMsg() uint64 SkipMsgs(seq uint64, num uint64) error LoadMsg(seq uint64, sm *StoreMsg) (*StoreMsg, error) diff --git a/server/store_test.go b/server/store_test.go index a916ceedb89..34619bfbf7a 100644 --- a/server/store_test.go +++ b/server/store_test.go @@ -57,7 +57,7 @@ func TestStoreMsgLoadNextMsgMulti(t *testing.T) { // Put 1k msgs in for i := 0; i < 1000; i++ { subj := fmt.Sprintf("foo.%d", i) - fs.StoreMsg(subj, nil, []byte("ZZZ")) + fs.StoreMsg(subj, nil, []byte("ZZZ"), 0) } var smv StoreMsg @@ -168,7 +168,7 @@ func TestStoreSubjectStateConsistency(t *testing.T) { // Publish an initial batch of messages. for i := 0; i < 4; i++ { - _, _, err := fs.StoreMsg("foo", nil, nil) + _, _, err := fs.StoreMsg("foo", nil, nil, 0) require_NoError(t, err) } @@ -221,7 +221,7 @@ func TestStoreSubjectStateConsistency(t *testing.T) { // Publish some more messages so we can test another scenario. for i := 0; i < 3; i++ { - _, _, err := fs.StoreMsg("foo", nil, nil) + _, _, err := fs.StoreMsg("foo", nil, nil, 0) require_NoError(t, err) } @@ -267,7 +267,7 @@ func TestStoreMaxMsgsPerUpdateBug(t *testing.T) { t, false, config(), func(t *testing.T, fs StreamStore) { for i := 0; i < 5; i++ { - _, _, err := fs.StoreMsg("foo", nil, nil) + _, _, err := fs.StoreMsg("foo", nil, nil, 0) require_NoError(t, err) } diff --git a/server/stream.go b/server/stream.go index 995d1293632..d7b701d94a3 100644 --- a/server/stream.go +++ b/server/stream.go @@ -100,6 +100,10 @@ type StreamConfig struct { // TODO(nat): Can/should we name these better? ConsumerLimits StreamConsumerLimits `json:"consumer_limits"` + // AllowMsgTTL allows header initiated per-message TTLs. If disabled, + // then the `NATS-TTL` header will be ignored. + AllowMsgTTL bool `json:"allow_msg_ttl"` + // Metadata is additional metadata for the Stream. Metadata map[string]string `json:"metadata,omitempty"` } @@ -413,6 +417,8 @@ const ( JSMsgRollup = "Nats-Rollup" JSMsgSize = "Nats-Msg-Size" JSResponseType = "Nats-Response-Type" + JSMessageTTL = "Nats-TTL" + JSMessageNoExpire = "Nats-No-Expire" ) // Headers for republished messages and direct gets. @@ -711,6 +717,7 @@ func (a *Account) addStreamWithAssignment(config *StreamConfig, fsConfig *FileSt fsCfg.SyncInterval = s.getOpts().SyncInterval fsCfg.SyncAlways = s.getOpts().SyncAlways fsCfg.Compression = config.Compression + fsCfg.EnforceTTLs = config.AllowMsgTTL if err := mset.setupStore(fsCfg); err != nil { mset.stop(true, false) @@ -1775,6 +1782,11 @@ func (jsa *jsAccount) configUpdateCheck(old, new *StreamConfig, s *Server, pedan } } + // Check on the allowed message TTL status. + if cfg.AllowMsgTTL != old.AllowMsgTTL { + return nil, NewJSStreamInvalidConfigError(fmt.Errorf("message TTL status can not be changed after stream creation")) + } + // Do some adjustments for being sealed. // Pedantic mode will allow those changes to be made, as they are determinictic and important to get a sealed stream. if cfg.Sealed { @@ -4194,6 +4206,30 @@ func getExpectedLastSeqPerSubjectForSubject(hdr []byte) string { return string(getHeader(JSExpectedLastSubjSeqSubj, hdr)) } +// Fast lookup of the message TTL: +// - Positive return value: duration in seconds. +// - Zero return value: no TTL or parse error. +// - Negative return value: don't expire. +func getMessageTTL(hdr []byte) (int64, error) { + ttl := getHeader(JSMessageTTL, hdr) + if len(ttl) == 0 { + return 0, nil + } + sttl := bytesToString(ttl) + dur, err := time.ParseDuration(sttl) + if err == nil { + if dur < time.Second { + return 0, NewJSMessageTTLInvalidError() + } + return int64(dur.Seconds()), nil + } + t := parseInt64(ttl) + if t < 0 { + return 0, NewJSMessageTTLInvalidError() + } + return t, nil +} + // Signal if we are clustered. Will acquire rlock. func (mset *stream) IsClustered() bool { mset.mu.RLock() @@ -5001,9 +5037,22 @@ func (mset *stream) processJetStreamMsg(subject, reply string, hdr, msg []byte, } } + // Find the message TTL if any. + ttl, err := getMessageTTL(hdr) + if err != nil { + if canRespond { + resp.PubAck = &PubAck{Stream: name} + resp.Error = NewJSMessageTTLInvalidError() + response, _ = json.Marshal(resp) + mset.outq.send(newJSPubMsg(reply, _EMPTY_, _EMPTY_, nil, response, nil, 0)) + } + mset.mu.Unlock() + return err + } + // Store actual msg. if lseq == 0 && ts == 0 { - seq, ts, err = store.StoreMsg(subject, hdr, msg) + seq, ts, err = store.StoreMsg(subject, hdr, msg, ttl) } else { // Make sure to take into account any message assignments that we had to skip (clfs). seq = lseq + 1 - clfs @@ -5011,7 +5060,7 @@ func (mset *stream) processJetStreamMsg(subject, reply string, hdr, msg []byte, if mset.hasAllPreAcks(seq, subject) { mset.clearAllPreAcks(seq) } - err = store.StoreRawMsg(subject, hdr, msg, seq, ts) + err = store.StoreRawMsg(subject, hdr, msg, seq, ts, ttl) } if err != nil { diff --git a/server/thw/thw.go b/server/thw/thw.go index 6fef7fbfdd3..5a544701049 100644 --- a/server/thw/thw.go +++ b/server/thw/thw.go @@ -216,6 +216,9 @@ func (hw *HashWheel) Encode(highSeq uint64) []byte { b = binary.LittleEndian.AppendUint64(b, hw.count) // Entry count b = binary.LittleEndian.AppendUint64(b, highSeq) // Stamp for _, slot := range hw.wheel { + if slot == nil || slot.entries == nil { + continue + } for v, ts := range slot.entries { b = binary.AppendVarint(b, ts) b = binary.AppendUvarint(b, v) From 6ea9fe0e7ed3b8570c755fa2aee5b2a093f5dbca Mon Sep 17 00:00:00 2001 From: Neil Twigg Date: Fri, 20 Dec 2024 16:45:55 +0000 Subject: [PATCH 2/2] Remove `EnforceTTLs` from filestore config; when `MaxAge` is set, use that when checking for next expiry Signed-off-by: Neil Twigg --- server/filestore.go | 14 +++++++++----- server/filestore_test.go | 28 ++++++++++++++-------------- server/stream.go | 2 -- 3 files changed, 23 insertions(+), 21 deletions(-) diff --git a/server/filestore.go b/server/filestore.go index f9fd375ce59..7544c0ea54b 100644 --- a/server/filestore.go +++ b/server/filestore.go @@ -69,8 +69,6 @@ type FileStoreConfig struct { Cipher StoreCipher // Compression is the algorithm to use when compressing. Compression StoreCompression - // EnforceTTLs decides whether or not to enforce per-message TTLs. - EnforceTTLs bool // Internal reference to our server. srv *Server @@ -418,7 +416,7 @@ func newFileStoreWithCreated(fcfg FileStoreConfig, cfg StreamConfig, created tim } // Only create a THW if we're going to allow TTLs. - if fs.fcfg.EnforceTTLs { + if cfg.AllowMsgTTL { fs.ttls = thw.NewHashWheel() } @@ -486,7 +484,7 @@ func newFileStoreWithCreated(fcfg FileStoreConfig, cfg StreamConfig, created tim } // See if we can bring back our TTL timed hash wheel state from disk. - if fcfg.EnforceTTLs { + if cfg.AllowMsgTTL { if err = fs.recoverTTLState(); err != nil && !os.IsNotExist(err) { fs.warn("Recovering TTL state from index errored: %v", err) } @@ -5324,7 +5322,13 @@ func (fs *fileStore) expireMsgs() { fs.ttls.ExpireTasks(func(seq uint64, ts int64) { fs.removeMsgViaLimits(seq) }) - nextTTL = fs.ttls.GetNextExpiration(math.MaxInt64) + if maxAge > 0 { + // Only check if we're expiring something in the next MaxAge interval, saves us a bit + // of work if MaxAge will beat us to the next expiry anyway. + nextTTL = fs.ttls.GetNextExpiration(time.Now().Add(time.Duration(maxAge)).UnixNano()) + } else { + nextTTL = fs.ttls.GetNextExpiration(math.MaxInt64) + } } // Onky cancel if no message left, not on potential lookup error that would result in sm == nil. diff --git a/server/filestore_test.go b/server/filestore_test.go index 6ca01973597..f78b69a3911 100644 --- a/server/filestore_test.go +++ b/server/filestore_test.go @@ -8299,8 +8299,8 @@ func TestFileStoreNumPendingMulti(t *testing.T) { func TestFileStoreMessageTTL(t *testing.T) { fs, err := newFileStore( - FileStoreConfig{StoreDir: t.TempDir(), EnforceTTLs: true}, - StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + FileStoreConfig{StoreDir: t.TempDir()}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage, AllowMsgTTL: true}) require_NoError(t, err) defer fs.Stop() @@ -8330,8 +8330,8 @@ func TestFileStoreMessageTTLRestart(t *testing.T) { t.Run("BeforeRestart", func(t *testing.T) { fs, err := newFileStore( - FileStoreConfig{StoreDir: dir, EnforceTTLs: true}, - StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + FileStoreConfig{StoreDir: dir}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage, AllowMsgTTL: true}) require_NoError(t, err) defer fs.Stop() @@ -8351,8 +8351,8 @@ func TestFileStoreMessageTTLRestart(t *testing.T) { t.Run("AfterRestart", func(t *testing.T) { fs, err := newFileStore( - FileStoreConfig{StoreDir: dir, EnforceTTLs: true}, - StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + FileStoreConfig{StoreDir: dir}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage, AllowMsgTTL: true}) require_NoError(t, err) defer fs.Stop() @@ -8379,8 +8379,8 @@ func TestFileStoreMessageTTLRecovered(t *testing.T) { t.Run("BeforeRestart", func(t *testing.T) { fs, err := newFileStore( - FileStoreConfig{StoreDir: dir, EnforceTTLs: true, srv: s}, - StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + FileStoreConfig{StoreDir: dir, srv: s}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage, AllowMsgTTL: true}) require_NoError(t, err) defer fs.Stop() @@ -8409,8 +8409,8 @@ func TestFileStoreMessageTTLRecovered(t *testing.T) { require_NoError(t, os.RemoveAll(fn)) fs, err := newFileStore( - FileStoreConfig{StoreDir: dir, EnforceTTLs: true, srv: s}, - StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + FileStoreConfig{StoreDir: dir, srv: s}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage, AllowMsgTTL: true}) require_NoError(t, err) defer fs.Stop() @@ -8437,8 +8437,8 @@ func TestFileStoreMessageTTLRecoveredSingleMessageWithoutStreamState(t *testing. t.Run("BeforeRestart", func(t *testing.T) { fs, err := newFileStore( - FileStoreConfig{StoreDir: dir, EnforceTTLs: true, srv: s}, - StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + FileStoreConfig{StoreDir: dir, srv: s}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage, AllowMsgTTL: true}) require_NoError(t, err) defer fs.Stop() @@ -8464,8 +8464,8 @@ func TestFileStoreMessageTTLRecoveredSingleMessageWithoutStreamState(t *testing. require_NoError(t, os.RemoveAll(fn)) fs, err := newFileStore( - FileStoreConfig{StoreDir: dir, EnforceTTLs: true, srv: s}, - StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage}) + FileStoreConfig{StoreDir: dir, srv: s}, + StreamConfig{Name: "zzz", Subjects: []string{"test"}, Storage: FileStorage, AllowMsgTTL: true}) require_NoError(t, err) defer fs.Stop() diff --git a/server/stream.go b/server/stream.go index d7b701d94a3..85c85868c63 100644 --- a/server/stream.go +++ b/server/stream.go @@ -717,7 +717,6 @@ func (a *Account) addStreamWithAssignment(config *StreamConfig, fsConfig *FileSt fsCfg.SyncInterval = s.getOpts().SyncInterval fsCfg.SyncAlways = s.getOpts().SyncAlways fsCfg.Compression = config.Compression - fsCfg.EnforceTTLs = config.AllowMsgTTL if err := mset.setupStore(fsCfg); err != nil { mset.stop(true, false) @@ -4209,7 +4208,6 @@ func getExpectedLastSeqPerSubjectForSubject(hdr []byte) string { // Fast lookup of the message TTL: // - Positive return value: duration in seconds. // - Zero return value: no TTL or parse error. -// - Negative return value: don't expire. func getMessageTTL(hdr []byte) (int64, error) { ttl := getHeader(JSMessageTTL, hdr) if len(ttl) == 0 {