From 4fa7ac26563c502937cea1dd5580d3dfc0c6a842 Mon Sep 17 00:00:00 2001 From: Edward Xiao Date: Wed, 12 Feb 2025 12:03:01 -0500 Subject: [PATCH] record: add WAL read ahead usage, new chunk format, and major version Add usages of WAL readAheadForCorruption to record.Next() and record.Read(). Write a new chunk format to include sync offsets for reading ahead. Add a new format major version for the new chunk format. --- format_major_version.go | 14 +- format_major_version_test.go | 8 +- open.go | 32 +- open_test.go | 2 +- record/log_writer.go | 97 ++- record/log_writer_test.go | 94 +++ record/record.go | 75 ++- record/record_test.go | 206 +++++- record/testdata/walSync | 1207 ++++++++++++++++++++++++++++++++++ testdata/checkpoint | 50 +- testdata/checkpoint_shared | 28 +- testdata/event_listener | 11 +- testdata/flushable_ingest | 14 +- tool/testdata/db_upgrade | 6 +- wal/failover_manager.go | 1 + wal/failover_writer.go | 4 + wal/standalone_manager.go | 7 +- wal/testdata/reader | 4 +- wal/wal.go | 4 + 19 files changed, 1764 insertions(+), 100 deletions(-) create mode 100644 record/testdata/walSync diff --git a/format_major_version.go b/format_major_version.go index 9812c48bd2..ad862d3db0 100644 --- a/format_major_version.go +++ b/format_major_version.go @@ -197,6 +197,13 @@ const ( // block. FormatColumnarBlocks + // FormatWALSyncChunks is a format major version enabling the writing of + // WAL sync chunks. These new chunks are used to disambiguate between corruption + // and logical EOF during WAL replay. This is implemented by adding a new + // chunk wire format that encodes an additional "Synced Offset" field which acts + // as a commitment that the WAL should have been synced up until the offset. + FormatWALSyncChunks + // -- Add new versions here -- // FormatNewest is the most recent format major version. @@ -235,7 +242,7 @@ func (v FormatMajorVersion) MaxTableFormat() sstable.TableFormat { case FormatDeleteSizedAndObsolete, FormatVirtualSSTables, FormatSyntheticPrefixSuffix, FormatFlushableIngestExcises: return sstable.TableFormatPebblev4 - case FormatColumnarBlocks: + case FormatColumnarBlocks, FormatWALSyncChunks: return sstable.TableFormatPebblev5 default: panic(fmt.Sprintf("pebble: unsupported format major version: %s", v)) @@ -248,7 +255,7 @@ func (v FormatMajorVersion) MinTableFormat() sstable.TableFormat { switch v { case FormatDefault, FormatFlushableIngest, FormatPrePebblev1MarkedCompacted, FormatDeleteSizedAndObsolete, FormatVirtualSSTables, FormatSyntheticPrefixSuffix, - FormatFlushableIngestExcises, FormatColumnarBlocks: + FormatFlushableIngestExcises, FormatColumnarBlocks, FormatWALSyncChunks: return sstable.TableFormatPebblev1 default: panic(fmt.Sprintf("pebble: unsupported format major version: %s", v)) @@ -291,6 +298,9 @@ var formatMajorVersionMigrations = map[FormatMajorVersion]func(*DB) error{ FormatColumnarBlocks: func(d *DB) error { return d.finalizeFormatVersUpgrade(FormatColumnarBlocks) }, + FormatWALSyncChunks: func(d *DB) error { + return d.finalizeFormatVersUpgrade(FormatWALSyncChunks) + }, } const formatVersionMarkerName = `format-version` diff --git a/format_major_version_test.go b/format_major_version_test.go index 78c1024432..5da290175f 100644 --- a/format_major_version_test.go +++ b/format_major_version_test.go @@ -25,11 +25,12 @@ func TestFormatMajorVersionStableValues(t *testing.T) { require.Equal(t, FormatVirtualSSTables, FormatMajorVersion(16)) require.Equal(t, FormatSyntheticPrefixSuffix, FormatMajorVersion(17)) require.Equal(t, FormatFlushableIngestExcises, FormatMajorVersion(18)) + require.Equal(t, FormatColumnarBlocks, FormatMajorVersion(19)) // When we add a new version, we should add a check for the new version in // addition to updating these expected values. - require.Equal(t, FormatNewest, FormatMajorVersion(19)) - require.Equal(t, internalFormatNewest, FormatMajorVersion(19)) + require.Equal(t, FormatNewest, FormatMajorVersion(20)) + require.Equal(t, internalFormatNewest, FormatMajorVersion(20)) } func TestFormatMajorVersion_MigrationDefined(t *testing.T) { @@ -60,6 +61,8 @@ func TestRatchetFormat(t *testing.T) { require.Equal(t, FormatFlushableIngestExcises, d.FormatMajorVersion()) require.NoError(t, d.RatchetFormatMajorVersion(FormatColumnarBlocks)) require.Equal(t, FormatColumnarBlocks, d.FormatMajorVersion()) + require.NoError(t, d.RatchetFormatMajorVersion(FormatWALSyncChunks)) + require.Equal(t, FormatWALSyncChunks, d.FormatMajorVersion()) require.NoError(t, d.Close()) @@ -217,6 +220,7 @@ func TestFormatMajorVersions_TableFormat(t *testing.T) { FormatSyntheticPrefixSuffix: {sstable.TableFormatPebblev1, sstable.TableFormatPebblev4}, FormatFlushableIngestExcises: {sstable.TableFormatPebblev1, sstable.TableFormatPebblev4}, FormatColumnarBlocks: {sstable.TableFormatPebblev1, sstable.TableFormatPebblev5}, + FormatWALSyncChunks: {sstable.TableFormatPebblev1, sstable.TableFormatPebblev5}, } // Valid versions. diff --git a/open.go b/open.go index 5423e8e6a8..2c7d5db42a 100644 --- a/open.go +++ b/open.go @@ -366,6 +366,7 @@ func Open(dirname string, opts *Options) (db *DB, err error) { QueueSemChan: d.commit.logSyncQSem, Logger: opts.Logger, EventListener: walEventListenerAdaptor{l: opts.EventListener}, + WriteWALSyncOffsets: FormatMajorVersion(d.mu.formatVers.vers.Load()) >= FormatWALSyncChunks, } if opts.WALFailover != nil { walOpts.Secondary = opts.WALFailover.Secondary @@ -930,18 +931,31 @@ func (d *DB) replayWAL( } if err != nil { // It is common to encounter a zeroed or invalid chunk due to WAL - // preallocation and WAL recycling. We need to distinguish these - // errors from EOF in order to recognize that the record was - // truncated and to avoid replaying subsequent WALs, but want - // to otherwise treat them like EOF. - if err == io.EOF { + // preallocation and WAL recycling. However zeroed or invalid chunks + // can also be a consequence of corruption / disk rot. When the log + // reader encounters one of these cases, it attempts to disambiguate + // by reading ahead looking for a future record. If a future chunk + // indicates the chunk at the original offset should've been valid, it + // surfaces record.ErrInvalidChunk or record.ErrZeroedChunk. These + // errors are always indicative of corruption and data loss. + // + // Otherwise, the reader surfaces io.ErrUnexpectedEOF indicating that + // the WAL terminated uncleanly and ambiguously. If the WAL is the + // most recent logical WAL, the caller passes in (strictWALTail=false), + // indicating we should tolerate the unclean ending. If the WAL is an + // older WAL, the caller passes in (strictWALTail=true), indicating that + // the WAL should have been closed cleanly, and we should interpret + // the `io.ErrUnexpectedEOF` as corruption and stop recovery. + if errors.Is(err, io.EOF) { break - } else if record.IsInvalidRecord(err) { - if !strictWALTail { - break - } + } else if errors.Is(err, io.ErrUnexpectedEOF) && !strictWALTail { + break + } else if errors.Is(err, record.ErrInvalidChunk) || errors.Is(err, record.ErrZeroedChunk) { + // If a read-ahead returns one of these errors, they should be marked with corruption. + // Other I/O related errors should not be marked with corruption and simply returned. err = errors.Mark(err, ErrCorruption) } + return nil, 0, errors.Wrap(err, "pebble: error when replaying WAL") } diff --git a/open_test.go b/open_test.go index 59acc66941..8b9653ac87 100644 --- a/open_test.go +++ b/open_test.go @@ -332,7 +332,7 @@ func TestNewDBFilenames(t *testing.T) { "LOCK", "MANIFEST-000001", "OPTIONS-000003", - "marker.format-version.000006.019", + "marker.format-version.000007.020", "marker.manifest.000001.MANIFEST-000001", }, } diff --git a/record/log_writer.go b/record/log_writer.go index 8ea1e17a6b..f9b8e35f36 100644 --- a/record/log_writer.go +++ b/record/log_writer.go @@ -5,6 +5,7 @@ package record import ( + "bytes" "context" "encoding/binary" "io" @@ -49,6 +50,19 @@ const ( SyncConcurrency = 1 << syncConcurrencyBits ) +const ( + + // noOpBytes is a byte written to help record.Reader distinguish between + // the recyclable format and the sync format when there is between 11 and 18 + // bytes remaining in a block. + noOpByte = byte(255) + + // noOpSequenceLength is a length of 4 bytes to be written to the zeroed header + // to distinguish when there is the special case of having between 11 and 18 bytes + // remaining in a block. + noOpSequenceLength = 4 +) + type syncSlot struct { wg *sync.WaitGroup err *error @@ -473,6 +487,17 @@ type LogWriter struct { pendingSyncsBackingIndex pendingSyncsWithHighestSyncIndex pendingSyncForSyncQueueBacking pendingSyncForSyncQueue + + // syncedOffset is the offset in the log that is durably synced after a + // flush. This member is used to write the WAL Sync chunk format's "Offset" + // field in the header. + syncedOffset atomic.Uint64 + + // emitFragment is set at runtime depending on which FormatMajorVersion + // is used. emitFragment will be set to writing WAL Sync chunk formats + // if the FormatMajorVersion is greater than or equal to FormatWALSyncChunks, + // otherwise it will write the recyclable chunk format. + emitFragment func(n int, p []byte) (remainingP []byte) } // LogWriterConfig is a struct used for configuring new LogWriters @@ -497,6 +522,9 @@ type LogWriterConfig struct { // package) precede the lower layer locks (in the record package). These // callbacks are serialized since they are invoked from the flushLoop. ExternalSyncQueueCallback ExternalSyncQueueCallback + + // WriteWALSyncOffsets represents whether to write the WAL sync chunk format. + WriteWALSyncOffsets bool } // ExternalSyncQueueCallback is to be run when a PendingSync has been @@ -537,6 +565,13 @@ func NewLogWriter( return time.AfterFunc(d, f) }, } + + if logWriterConfig.WriteWALSyncOffsets { + r.emitFragment = r.emitFragmentSyncOffsets + } else { + r.emitFragment = r.emitFragmentRecyclable + } + m := &LogWriterMetrics{} if logWriterConfig.ExternalSyncQueueCallback != nil { r.pendingSyncsBackingIndex.init(logWriterConfig.ExternalSyncQueueCallback) @@ -584,6 +619,11 @@ func (w *LogWriter) flushLoop(context.Context) { f.Unlock() }() + // writtenOffset is the amount of data that has been written + // but not necessarily synced. This is used to update logWriter's + // syncedOffset after a sync. + var writtenOffset uint64 = 0 + // The flush loop performs flushing of full and partial data blocks to the // underlying writer (LogWriter.w), syncing of the writer, and notification // to sync requests that they have completed. @@ -694,9 +734,11 @@ func (w *LogWriter) flushLoop(context.Context) { f.Lock() continue } + writtenOffset += uint64(len(data)) synced, syncLatency, bytesWritten, err := w.flushPending(data, pending, snap) f.Lock() if synced && f.fsyncLatency != nil { + w.syncedOffset.Store(writtenOffset) f.fsyncLatency.Observe(float64(syncLatency)) } f.err = err @@ -954,6 +996,11 @@ func (w *LogWriter) Size() int64 { return w.blockNum*blockSize + int64(w.block.written.Load()) } +// emitEOFTrailer writes a special recyclable chunk header to signal EOF. +// The reason why this function writes the recyclable chunk header instead +// of having a function for writing recyclable and WAL sync chunks as +// emitFragment does it because there is no reason to add 8 additional +// bytes to the EOFTrailer for the SyncedOffset as it will be zeroed out anyway. func (w *LogWriter) emitEOFTrailer() { // Write a recyclable chunk header with a different log number. Readers // will treat the header as EOF when the log number does not match. @@ -966,7 +1013,7 @@ func (w *LogWriter) emitEOFTrailer() { b.written.Store(i + int32(recyclableHeaderSize)) } -func (w *LogWriter) emitFragment(n int, p []byte) (remainingP []byte) { +func (w *LogWriter) emitFragmentRecyclable(n int, p []byte) (remainingP []byte) { b := w.block i := b.written.Load() first := n == 0 @@ -1003,6 +1050,54 @@ func (w *LogWriter) emitFragment(n int, p []byte) (remainingP []byte) { return p[r:] } +func (w *LogWriter) emitFragmentSyncOffsets(n int, p []byte) (remainingP []byte) { + b := w.block + i := b.written.Load() + first := n == 0 + last := blockSize-i-walSyncHeaderSize >= int32(len(p)) + + if last { + if first { + b.buf[i+6] = walSyncFullChunkEncoding + } else { + b.buf[i+6] = walSyncLastChunkEncoding + } + } else { + if first { + b.buf[i+6] = walSyncFirstChunkEncoding + } else { + b.buf[i+6] = walSyncMiddleChunkEncoding + } + } + + binary.LittleEndian.PutUint32(b.buf[i+7:i+11], w.logNum) + binary.LittleEndian.PutUint64(b.buf[i+11:i+19], w.syncedOffset.Load()) + + r := copy(b.buf[i+walSyncHeaderSize:], p) + j := i + int32(walSyncHeaderSize+r) + binary.LittleEndian.PutUint32(b.buf[i+0:i+4], crc.New(b.buf[i+6:j]).Value()) + binary.LittleEndian.PutUint16(b.buf[i+4:i+6], uint16(r)) + b.written.Store(j) + + if blockSize-b.written.Load() < walSyncHeaderSize { + // There is no room for another fragment in the block, so fill the + // remaining bytes with zeros and queue the block for flushing. + clear(b.buf[b.written.Load():]) + + // When there is more than 11 bytes (recyclableHeaderSize) and strictly + // less than 19 bytes (walSyncHeaderSize) remaining, we need to mark + // this special case so that the reader will be able to correctly interpret + // what to do when it sees a zeroed header. The no-op sequence is written + // in the next 4 bytes of what would have been the header. + if blockSize-b.written.Load() >= recyclableHeaderSize { + noOpSequence := bytes.Repeat([]byte{noOpByte}, noOpSequenceLength) + copy(b.buf[b.written.Load()+7:], noOpSequence) + } + w.queueBlock() + } + return p[r:] +} + // Metrics must typically be called after Close, since the callee will no // longer modify the returned LogWriterMetrics. It is also current if there is // nothing left to flush in the flush loop, but that is an implementation diff --git a/record/log_writer_test.go b/record/log_writer_test.go index 5cf740b4ae..b376951ce5 100644 --- a/record/log_writer_test.go +++ b/record/log_writer_test.go @@ -6,10 +6,13 @@ package record import ( "bytes" + "encoding/binary" "fmt" + "io" "math" "math/rand/v2" "sort" + "strings" "sync" "sync/atomic" "testing" @@ -171,11 +174,13 @@ func TestSyncError(t *testing.T) { type syncFile struct { writePos atomic.Int64 syncPos atomic.Int64 + buffer bytes.Buffer } func (f *syncFile) Write(buf []byte) (int, error) { n := len(buf) f.writePos.Add(int64(n)) + f.buffer.Write(buf) return n, nil } @@ -733,6 +738,95 @@ func TestSyncRecordGeneralizedWithCloseError(t *testing.T) { require.Less(t, f.syncPos.Load(), f.writePos.Load()) } +func writeWALSyncRecords(t *testing.T, numRecords int, recordSizes []int) *syncFile { + f := &syncFile{} + w := NewLogWriter(f, 1, LogWriterConfig{WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}), WriteWALSyncOffsets: true}) + var syncErr error + for i := 0; i < numRecords; i++ { + var syncWG sync.WaitGroup + syncWG.Add(1) + data := []byte(strings.Repeat(fmt.Sprintf("%d", i%10), recordSizes[i])) + offset, err := w.SyncRecord(data, &syncWG, &syncErr) + require.NoError(t, err) + syncWG.Wait() + require.NoError(t, syncErr) + if v := f.writePos.Load(); offset != v { + t.Fatalf("expected write pos %d, but found %d", offset, v) + } + if v := f.syncPos.Load(); offset != v { + t.Fatalf("expected sync pos %d, but found %d", offset, v) + } + } + return f +} + +func validateWALSyncRecords(t *testing.T, buf *bytes.Buffer) { + var largestOffset uint64 = 0 + i := 0 + bufBytes := (*buf).Bytes() + for i < len(bufBytes) { + if blockSize-(i%blockSize) < walSyncHeaderSize { + i += blockSize - (i % blockSize) + continue + } + + checksum := binary.LittleEndian.Uint32(bufBytes[i+0 : i+4]) + length := binary.LittleEndian.Uint16(bufBytes[i+4 : i+6]) + chunkEncoding := bufBytes[i+6] + logNum := binary.LittleEndian.Uint32(bufBytes[i+7 : i+11]) + + // Reader and Writer have a logNum of 1, so a logNum of 2 is EOF. + if logNum == 2 { + // reached EOF trailer + if checksum != 0 && length != 0 { + t.Fatal("Mismatched logNum but not EOF trailer") + } + break + } + offset := binary.LittleEndian.Uint64(bufBytes[i+11 : i+19]) + if offset < largestOffset { + t.Fatal("Expected monotonitcally increasing offsets.") + } + largestOffset = offset + headerFormat := headerFormatMappings[chunkEncoding] + headerSize := headerFormat.headerSize + i += headerSize + int(length) + } + + r := NewReader(bytes.NewBuffer(bufBytes), 1) + for { + rr, err := r.Next() + if err == io.EOF { + break + } + if err != nil { + t.Fatalf("Error Next(): %v", err) + } + _, err = io.ReadAll(rr) + if err != nil { + t.Fatalf("Error Next(): %v", err) + } + } +} + +func TestWALSyncOffsetSimple(t *testing.T) { + recordSizes := []int{} + for i := 0; i < 1000; i++ { + recordSizes = append(recordSizes, blockSize-walSyncHeaderSize) + } + f := writeWALSyncRecords(t, len(recordSizes), recordSizes) + validateWALSyncRecords(t, &f.buffer) +} + +func TestWALSyncOffsetRandom(t *testing.T) { + recordSizes := []int{} + for i := 0; i < 1000; i++ { + recordSizes = append(recordSizes, 1+rand.IntN(blockSize-walSyncHeaderSize)) + } + f := writeWALSyncRecords(t, len(recordSizes), recordSizes) + validateWALSyncRecords(t, &f.buffer) +} + // BenchmarkQueueWALBlocks exercises queueing within the LogWriter. It can be // useful to measure allocations involved when flushing is slow enough to // accumulate a large backlog fo queued blocks. diff --git a/record/record.go b/record/record.go index e5065eb2f0..fd907dfc21 100644 --- a/record/record.go +++ b/record/record.go @@ -276,26 +276,62 @@ func (r *Reader) nextChunk(wantFirst bool) error { chunkEncoding := r.buf[r.end+6] if int(chunkEncoding) >= len(headerFormatMappings) { + r.invalidOffset = uint64(r.blockNum)*blockSize + uint64(r.begin) return ErrInvalidChunk } headerFormat := headerFormatMappings[chunkEncoding] chunkPosition, wireFormat, headerSize := headerFormat.chunkPosition, headerFormat.wireFormat, headerFormat.headerSize if checksum == 0 && length == 0 && chunkPosition == invalidChunkPosition { + // remaining bytes < 11 + // The remaining bytes in the block is < 11 so regardless of which chunk format is + // being written (Recyclable or walSync), we should skip to the next block. if r.end+recyclableHeaderSize > r.n { // Skip the rest of the block if the recyclable header size does not - // fit within it. + // fit within it. The end of a block will be zeroed out if the log writer + // cannot fit another chunk into it, even a chunk with no payload like + // the EOF Trailer. r.end = r.n continue } + + // 11 <= remaining bytes < 19 + // The remaining bytes in the block can fit a recyclable header but not a walSync + // header size. In this case, scan for the no-op sequence. + // + // If the no-op sequence is found, then the walSync format was being written but + // there was not enough space for another chunk; we should skip to the next block + // in the log. + // + // However, if the no-op sequence is not found, then the recyclable format was being + // written. Because there was enough space to write a recyclable header but a + // zeroed 6 bytes were found, ErrZeroedChunk should be returned. + if r.end+walSyncHeaderSize > r.n { + readingWALSyncFormat := true + for i := 0; i < noOpSequenceLength && readingWALSyncFormat; i++ { + if r.buf[r.end+7+i] != noOpByte { + readingWALSyncFormat = false + } + } + if readingWALSyncFormat { + r.end = r.n + continue + } + } + + // The last case is when there was more than 19 bytes which means there shouldn't be + // a zeroed header. Thus, this case should also return ErrZeroedChunk. + r.invalidOffset = uint64(r.blockNum)*blockSize + uint64(r.begin) return ErrZeroedChunk } if wireFormat == invalidWireFormat { + r.invalidOffset = uint64(r.blockNum)*blockSize + uint64(r.begin) return ErrInvalidChunk } - if wireFormat == recyclableWireFormat { + if wireFormat == recyclableWireFormat || wireFormat == walSyncWireFormat { if r.end+headerSize > r.n { + r.invalidOffset = uint64(r.blockNum)*blockSize + uint64(r.begin) return ErrInvalidChunk } @@ -308,6 +344,7 @@ func (r *Reader) nextChunk(wantFirst bool) error { } // Otherwise, treat this chunk as invalid in order to prevent reading // of a partial record. + r.invalidOffset = uint64(r.blockNum)*blockSize + uint64(r.begin) return ErrInvalidChunk } } @@ -316,9 +353,11 @@ func (r *Reader) nextChunk(wantFirst bool) error { r.end = r.begin + int(length) if r.end > r.n { // The chunk straddles a 32KB boundary (or the end of file). + r.invalidOffset = uint64(r.blockNum)*blockSize + uint64(r.begin) return ErrInvalidChunk } if checksum != crc.New(r.buf[r.begin-headerSize+6:r.end]).Value() { + r.invalidOffset = uint64(r.blockNum)*blockSize + uint64(r.begin) return ErrInvalidChunk } if wantFirst { @@ -334,6 +373,7 @@ func (r *Reader) nextChunk(wantFirst bool) error { // This can happen if the previous instance of the log ended with a // partial block at the same blockNum as the new log but extended // beyond the partial block of the new log. + r.invalidOffset = uint64(r.blockNum)*blockSize + uint64(r.begin) return ErrInvalidChunk } return io.EOF @@ -341,6 +381,7 @@ func (r *Reader) nextChunk(wantFirst bool) error { n, err := io.ReadFull(r.r, r.buf[:]) if err != nil && err != io.ErrUnexpectedEOF { if err == io.EOF && !wantFirst { + r.invalidOffset = uint64(r.blockNum)*blockSize + uint64(r.begin) return io.ErrUnexpectedEOF } return err @@ -360,11 +401,10 @@ func (r *Reader) Next() (io.Reader, error) { } r.begin = r.end r.err = r.nextChunk(true) - // TODO(edward) usage of readAheadForCorruption; uncomment in follow PR - // if r.err == ErrInvalidChunk || r.err == ErrZeroedChunk { - // readAheadResult := r.readAheadForCorruption() - // return nil, readAheadResult - // } + if errors.Is(r.err, ErrInvalidChunk) || errors.Is(r.err, ErrZeroedChunk) { + readAheadResult := r.readAheadForCorruption() + return nil, readAheadResult + } if r.err != nil { return nil, r.err } @@ -389,6 +429,9 @@ func (r *Reader) readAheadForCorruption() error { for { // Load the next block into r.buf. n, err := io.ReadFull(r.r, r.buf[:]) + r.begin, r.end, r.n = 0, 0, n + r.blockNum++ + if errors.Is(err, io.EOF) { // io.ErrUnexpectedEOF is returned instead of // io.EOF because io library functions clear @@ -411,9 +454,6 @@ func (r *Reader) readAheadForCorruption() error { return err } - r.begin, r.end, r.n = 0, 0, n - r.blockNum++ - for r.end+legacyHeaderSize <= r.n { checksum := binary.LittleEndian.Uint32(r.buf[r.end+0 : r.end+4]) length := binary.LittleEndian.Uint16(r.buf[r.end+4 : r.end+6]) @@ -431,7 +471,7 @@ func (r *Reader) readAheadForCorruption() error { break } if wireFormat == recyclableWireFormat || wireFormat == walSyncWireFormat { - if r.begin+headerSize > r.n { + if r.end+headerSize > r.n { break } logNum := binary.LittleEndian.Uint32(r.buf[r.end+7 : r.end+11]) @@ -533,15 +573,14 @@ func (x singleReader) Read(p []byte) (int, error) { if r.last { return 0, io.EOF } - if r.err = r.nextChunk(false); r.err != nil { + r.err = r.nextChunk(false) + if errors.Is(r.err, ErrInvalidChunk) || errors.Is(r.err, ErrZeroedChunk) { + readAheadResult := r.readAheadForCorruption() + return 0, readAheadResult + } + if r.err != nil { return 0, r.err } - // TODO(edward) usage of readAheadForCorruption; uncomment in follow PR - // r.err = r.nextChunk(false) - // if r.err == ErrInvalidChunk || r.err == ErrZeroedChunk { - // readAheadResult := r.readAheadForCorruption() - // return 0, readAheadResult - // } } n := copy(p, r.buf[r.begin:r.end]) r.begin += n diff --git a/record/record_test.go b/record/record_test.go index e2342e57b7..87fe33e738 100644 --- a/record/record_test.go +++ b/record/record_test.go @@ -11,11 +11,17 @@ import ( "io" "math" "math/rand/v2" + "slices" "strings" "testing" "time" + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/internal/binfmt" + "github.com/cockroachdb/pebble/internal/crc" + "github.com/cockroachdb/pebble/internal/treeprinter" "github.com/prometheus/client_golang/prometheus" "github.com/stretchr/testify/require" ) @@ -431,7 +437,7 @@ func TestCorruptBlock(t *testing.T) { if err == nil { t.Fatal("Expected a checksum mismatch error, got nil") } - if err != ErrInvalidChunk { + if err != io.ErrUnexpectedEOF { t.Fatalf("Unexpected error returned: %v", err) } } @@ -663,8 +669,8 @@ func TestInvalidLogNum(t *testing.T) { { r := NewReader(bytes.NewReader(buf.Bytes()), 2) - if _, err := r.Next(); err != ErrInvalidChunk { - t.Fatalf("expected %s, but found %s\n", ErrInvalidChunk, err) + if _, err := r.Next(); err != io.ErrUnexpectedEOF { + t.Fatalf("expected %s, but found %s\n", io.ErrUnexpectedEOF, err) } } } @@ -747,7 +753,7 @@ func TestRecycleLog(t *testing.T) { rr, err := r.Next() if err != nil { // If we limited output then an EOF, zeroed, or invalid chunk is expected. - if limitedBuf.limit < 0 && (err == io.EOF || err == ErrZeroedChunk || err == ErrInvalidChunk) { + if limitedBuf.limit < 0 && (err == io.EOF || err == io.ErrUnexpectedEOF || err == ErrZeroedChunk || err == ErrInvalidChunk) { break } t.Fatalf("%d/%d: %v", i, j, err) @@ -755,7 +761,7 @@ func TestRecycleLog(t *testing.T) { x, err := io.ReadAll(rr) if err != nil { // If we limited output then an EOF, zeroed, or invalid chunk is expected. - if limitedBuf.limit < 0 && (err == io.EOF || err == ErrZeroedChunk || err == ErrInvalidChunk) { + if limitedBuf.limit < 0 && (err == io.EOF || err == io.ErrUnexpectedEOF || err == ErrZeroedChunk || err == ErrInvalidChunk) { break } t.Fatalf("%d/%d: %v", i, j, err) @@ -764,7 +770,7 @@ func TestRecycleLog(t *testing.T) { t.Fatalf("%d/%d: expected record %d, but found %d", i, j, sizes[j], len(x)) } } - if _, err := r.Next(); err != io.EOF && err != ErrZeroedChunk && err != ErrInvalidChunk { + if _, err := r.Next(); err != io.EOF && err != io.ErrUnexpectedEOF && err != ErrZeroedChunk && err != ErrInvalidChunk { t.Fatalf("%d: expected EOF, but found %v", i, err) } } @@ -875,7 +881,186 @@ func TestRecycleLogWithPartialRecord(t *testing.T) { require.NoError(t, err) _, err = io.ReadAll(rr) - require.Equal(t, err, ErrInvalidChunk) + require.Equal(t, err, io.ErrUnexpectedEOF) +} + +func TestWALSync(t *testing.T) { + var buffer bytes.Buffer + result := make([]byte, 0) + corruptChunkNumbers := make([]int, 0) + + datadriven.RunTest(t, "testdata/walSync", func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "init": + buffer.Reset() + result = result[:0] + + for chunkNumber, line := range strings.Split(d.Input, "\n") { + switch { + case strings.HasPrefix(line, "writeChunk"): + var encodedLength, chunkLength uint16 + var logNum uint32 + var offset uint64 + var encoding uint8 + var corrupt bool + + _, err := fmt.Sscanf(line, "writeChunk encodedLength=%d chunkLength=%d encoding=%d logNum=%d offset=%d corrupt=%t", + &encodedLength, &chunkLength, &encoding, &logNum, &offset, &corrupt) + if err != nil { + return fmt.Sprintf("error parsing length line: %v", err) + } + + chunk := make([]byte, walSyncHeaderSize+chunkLength) + + binary.LittleEndian.PutUint16(chunk[4:6], encodedLength) + chunk[6] = encoding + binary.LittleEndian.PutUint32(chunk[7:11], logNum) + binary.LittleEndian.PutUint64(chunk[11:19], offset) + + checksum := uint32(crc.New(chunk[6:]).Value()) + binary.LittleEndian.PutUint32(chunk[0:4], checksum) + + if corrupt { + for i := 0; i < 4; i++ { + chunk[i] ^= 1 + } + corruptChunkNumbers = append(corruptChunkNumbers, chunkNumber) + } + + buffer.Write(chunk) + + case strings.HasPrefix(line, "endblock"): + remainingBytes := buffer.Len() % blockSize + buffer.Write(make([]byte, remainingBytes)) + + case strings.HasPrefix(line, "EOF"): + var logNum uint32 + _, err := fmt.Sscanf(line, "EOF logNum=%d", &logNum) + if err != nil { + return fmt.Sprintf("error parsing EOF line: %v", err) + } + + eofChunk := make([]byte, 19) + eofChunk[6] = walSyncFullChunkEncoding + binary.LittleEndian.PutUint32(eofChunk[7:11], logNum) + buffer.Write(eofChunk) + + case strings.HasPrefix(line, "raw"): + rawChunk, err := parseRawChunk(line) + if err != nil { + return fmt.Sprintf("error parsing raw line: %v", err) + } + buffer.Write(rawChunk) + } + } + + case "read": + r := NewReader(bytes.NewBuffer(buffer.Bytes()), 1) + + for { + reader, err := r.Next() + if err != nil { + return fmt.Sprintf("error reading next: %v\nfinal blockNum: %d\nbytes read: %d", err, r.blockNum, len(result)) + } + + data, err := io.ReadAll(reader) + if err != nil { + return fmt.Sprintf("error reading all: %v\nfinal blockNum: %d\nbytes read: %d", err, r.blockNum, len(result)) + } + result = append(result, data...) + } + + case "describe": + formatter := binfmt.New(buffer.Bytes()).LineWidth(20) + tree := treeprinter.New() + describeWALSyncBlocks(formatter, &tree, buffer.Bytes(), corruptChunkNumbers) + return tree.String() + } + + return "" + }) +} + +func parseRawChunk(input string) ([]byte, error) { + words := strings.Fields(input) + var result []byte + for _, word := range words { + var value byte + if strings.HasPrefix(word, "0x") { + _, err := fmt.Sscanf(word, "0x%02X", &value) + if err != nil { + return nil, errors.Errorf("failed to parse hex: %s", word) + } + result = append(result, value) + } else if word == "#" { + break + } + } + return result, nil +} + +// describeWALSyncBlocks is used to visualize blocks and chunks with the assumption +// that they are generally well formed. Having one of the corruption conditions +// may cause the visualization to be incorrect due to the assumption of well-formedness. +func describeWALSyncBlocks( + f *binfmt.Formatter, tp *treeprinter.Node, data []byte, corruptChunks []int, +) { + i := 0 + n := tp.Child("Blocks") + globalChunkNumber := 0 + + for i < len(data) { + if i%blockSize == 0 { + blockNum := i / blockSize + + blockNode := n.Childf("Block #%d", blockNum) + + chunkNumber := 0 + for i < len(data) { + var chunkNode treeprinter.Node + if slices.Contains(corruptChunks, globalChunkNumber) { + chunkNode = blockNode.Childf("Chunk #%d (offset %d, corrupt)", chunkNumber, i) + } else { + chunkNode = blockNode.Childf("Chunk #%d (offset %d)", chunkNumber, i) + } + + checksum := binary.LittleEndian.Uint32(data[i+0 : i+4]) + length := binary.LittleEndian.Uint16(data[i+4 : i+6]) + + if int(length) == 0 { + chunkNode.Child("EOF") + f.SetAnchorOffset() + f.ToTreePrinter(n) + return + } + + chunkEncoding := data[i+6] + headerFormat := headerFormatMappings[chunkEncoding] + chunkType, wireFormat, headerSize := headerFormat.chunkPosition, headerFormat.wireFormat, headerFormat.headerSize + + logNum := binary.LittleEndian.Uint32(data[i+7 : i+11]) + offset := binary.LittleEndian.Uint64(data[i+11 : i+19]) + + chunkNode.Childf("Checksum: %d", checksum) + chunkNode.Childf("Encoded Length: %d", length) + chunkNode.Childf("Chunk encoding: %d (chunkType: %d, wireFormat: %d)", chunkEncoding, chunkType, wireFormat) + chunkNode.Childf("Log Num: %d", logNum) + chunkNode.Childf("Synced Offset: %d", offset) + + i += headerSize + int(length) + chunkNumber++ + globalChunkNumber++ + + if i%blockSize == 0 { + break + } + + } + } + } + + f.SetAnchorOffset() + f.ToTreePrinter(n) } func BenchmarkRecordWrite(b *testing.B) { @@ -897,10 +1082,3 @@ func BenchmarkRecordWrite(b *testing.B) { }) } } - -// TODO(edward) Suppresses linting warning. -// Delete after readAheadForCorruption() is called in follow-up PRs. -func TestReadAheadForCorruption(t *testing.T) { - r := NewReader(new(bytes.Buffer), 0) - r.readAheadForCorruption() -} diff --git a/record/testdata/walSync b/record/testdata/walSync new file mode 100644 index 0000000000..dc28f4105e --- /dev/null +++ b/record/testdata/walSync @@ -0,0 +1,1207 @@ +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=false +EOF logNum=2 +---- + +describe +---- +Blocks + ├── Block #0 + │ ├── Chunk #0 (offset 0) + │ │ ├── Checksum: 3699662224 + │ │ ├── Encoded Length: 1 + │ │ ├── Chunk encoding: 9 (chunkType: 1, wireFormat: 3) + │ │ ├── Log Num: 1 + │ │ └── Synced Offset: 0 + │ └── Chunk #1 (offset 20) + │ └── EOF + └── + +read +---- +error reading next: EOF +final blockNum: 0 +bytes read: 1 + + +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=true +EOF logNum=2 +---- + +describe +---- +Blocks + ├── Block #0 + │ ├── Chunk #0 (offset 0, corrupt) + │ │ ├── Checksum: 3716504721 + │ │ ├── Encoded Length: 1 + │ │ ├── Chunk encoding: 9 (chunkType: 1, wireFormat: 3) + │ │ ├── Log Num: 1 + │ │ └── Synced Offset: 0 + │ └── Chunk #1 (offset 20) + │ └── EOF + └── + +read +---- +error reading next: unexpected EOF +final blockNum: 1 +bytes read: 0 + +########################### +##### CRC Error Cases ##### +########################### + +# Simple corruption in single block with no confirmation +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=true +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 1 +bytes read: 0 + +# Simple corruption spanning the entire block with no confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=true +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 2 +bytes read: 0 + +# Multiple corruption with no confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=true +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=true + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Multiple corruption but large offset is also corrupt, no confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=true +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=100000 corrupt=true + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Simple corruption with confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=true +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=30000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 0 + +# Corrupt the first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a confirmation chunk. Observe that final blockNum == 1 +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=true +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 0 + +# Corrupt the first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a corrupt chunk, leading to EOF +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=true +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=true +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Complex multiple corruption with no confirmation +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=true +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=true +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=32000 corrupt=true +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=32000 corrupt=true +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=64000 corrupt=true +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 5 +bytes read: 0 + +# Complex multiple corruption with confirmation in blockNum 3 +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=true +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=true +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=true +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=true +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 3 +bytes read: 0 + +# Complex multiple corruption with confirmation offset too small +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=true +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=true +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=true +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=100 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=100 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 5 +bytes read: 32749 + +########################### +###### log num cases ###### +########################### +# NOTE: reader logNum == 1 +########################### + + +# Simple logNum issues in single block with no confirmation +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=0 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 1 +bytes read: 0 + +# Simple logNum issues spanning the entire block with no confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=0 offset=0 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 2 +bytes read: 0 + +# Multiple logNum issues with no confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=0 offset=0 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=0 offset=0 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Multiple logNum issues but large offset is also has issues, no confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=0 offset=0 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=100 offset=100000 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Simple logNum issues with confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=0 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=30000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 0 + +# Issues in first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a confirmation chunk. Observe that final blockNum == 1 +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 0 + +# Issues in first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a logNum issue chunk, leading to EOF +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=0 offset=32000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Complex multiple logNum issues with no confirmation +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=0 offset=32000 corrupt=false + +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=32000 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=32000 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=32000 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=0 offset=64000 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 5 +bytes read: 0 + + +# Complex multiple logNum issues with confirmation in blockNum 3 +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=0 offset=32000 corrupt=false + +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 3 +bytes read: 0 + + +# Complex multiple logNum issue with confirmation offset too small +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=0 offset=0 corrupt=false + +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=100 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=100 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 5 +bytes read: 32749 + +####################################### +###### invalid wire format cases ###### +####################################### +# Encoding == 0 leads to a wireFormat error + + +# Simple wire issues in single block with no confirmation +init +writeChunk encodedLength=1 chunkLength=1 encoding=0 logNum=1 offset=0 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 1 +bytes read: 0 + +# Simple wire issues spanning the entire block with no confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=0 logNum=1 offset=0 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 2 +bytes read: 0 + +# Multiple wire issues with no confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=0 logNum=1 offset=0 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=0 logNum=1 offset=0 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Multiple wire issues but large offset is also has issues, no confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=0 logNum=1 offset=0 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=0 logNum=1 offset=100000 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Simple wire issues with confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=0 logNum=1 offset=0 corrupt=false + +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=30000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 0 + +# Issues in first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a confirmation chunk. Observe that final blockNum == 1 +init +writeChunk encodedLength=1 chunkLength=1 encoding=0 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 0 + +# Issues in first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a wire issue chunk, leading to EOF +init +writeChunk encodedLength=1 chunkLength=1 encoding=0 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=0 logNum=1 offset=32000 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Complex multiple wire issues with no confirmation +init +writeChunk encodedLength=1 chunkLength=1 encoding=0 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=0 logNum=1 offset=32000 corrupt=false + +writeChunk encodedLength=1 chunkLength=1 encoding=0 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=0 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=32000 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=0 logNum=1 offset=64000 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 5 +bytes read: 0 + + +# Complex multiple wire issues with confirmation in blockNum 3 +init +writeChunk encodedLength=1 chunkLength=1 encoding=0 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=0 logNum=1 offset=32000 corrupt=false + +writeChunk encodedLength=1 chunkLength=1 encoding=0 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 3 +bytes read: 0 + + +# Complex multiple wire issue with confirmation offset too small +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=0 logNum=1 offset=0 corrupt=false + +writeChunk encodedLength=1 chunkLength=1 encoding=0 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=0 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=100 corrupt=false + +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=100 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 5 +bytes read: 32749 + +########################### +###### zeroed chunk ###### +########################### +# Zeroed chunk has checksum, encodedLength, and chunkEncoding == 0; minimum 7 zeroes + +# zeroed chunk +init +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's + +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=60000 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: zeroed chunk +final blockNum: 1 +bytes read: 32738 + +# Simple zeroed issues in single block with no confirmation +init +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 2 +bytes read: 32738 + +# Multiple zeroed issues with no confirmation +init +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Multiple zeroed issues but large offset is also has issues, no confirmation +init +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=1000000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 32738 + +# Simple zeroed issues with confirmation +init +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's + +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=100000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: zeroed chunk +final blockNum: 1 +bytes read: 32738 + +init +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=100000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: zeroed chunk +final blockNum: 1 +bytes read: 0 + +# Issues in first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a confirmation chunk. Observe that final blockNum == 1 +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's +writeChunk encodedLength=32718 chunkLength=32718 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=1000000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: zeroed chunk +final blockNum: 1 +bytes read: 1 + +# Issues in first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a zeroed chunk, leading to EOF +init +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32718 chunkLength=32718 encoding=9 logNum=1 offset=60 corrupt=false +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's +writeChunk encodedLength=32738 chunkLength=32738 encoding=0 logNum=1 offset=32000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Multiple zeroed chunk issues with confirmation +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 +writeChunk encodedLength=32718 chunkLength=32718 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=1 corrupt=false +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 +writeChunk encodedLength=32718 chunkLength=32718 encoding=9 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=64000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: zeroed chunk +final blockNum: 2 +bytes read: 1 + +# Complex multiple zeroed chunks with confirmation offset too small +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's +writeChunk encodedLength=32718 chunkLength=32718 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=1 corrupt=false +raw 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 0x00 # 11 0x00's +writeChunk encodedLength=32718 chunkLength=32718 encoding=9 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=100 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 5 +bytes read: 32750 + +##################################### +###### encodedLength too long ###### +##################################### +# if r.end > r.n { + +init +writeChunk encodedLength=64000 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 0 + +# Simple encodedLength issues in single block with no confirmation +init +writeChunk encodedLength=64000 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 1 +bytes read: 0 + +# Simple encodedLength issues spanning the entire block with no confirmation +init +writeChunk encodedLength=64000 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 2 +bytes read: 0 + +# Multiple encodedLength issues with no confirmation +init +writeChunk encodedLength=64000 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=64000 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Multiple encodedLength issues but large offset is also has issues, no confirmation +init +writeChunk encodedLength=64000 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=64000 chunkLength=32749 encoding=9 logNum=1 offset=1000 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Simple encodedLength issues with confirmation +init +writeChunk encodedLength=64000 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=30000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 0 + +# Issues in first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a confirmation chunk. Observe that final blockNum == 1 +init +writeChunk encodedLength=200 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=1000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 0 + +# Issues in first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a encodedLength issue chunk, leading to EOF +init +writeChunk encodedLength=200 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=64000 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Complex multiple encodedLength issues with no confirmation +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=0 offset=32000 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=32000 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=0 offset=32000 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=0 offset=64000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 5 +bytes read: 0 + +# Complex multiple encodedLength issues with confirmation in blockNum 3 +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=200 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=64000 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=200 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 3 +bytes read: 1 + +# Complex multiple encodedLength issues with confirmation offset too small +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=200 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=64000 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=200 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=100 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 6 +bytes read: 32750 + +########################### +#### Encoding too Large ### +########################### +# if int(chunkEncoding) >= len(headerFormatMappings) { + +# Simple encoding issues in single block with no confirmation +init +writeChunk encodedLength=1 chunkLength=1 encoding=100 logNum=1 offset=0 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 1 +bytes read: 0 + +# Simple encoding issues spanning the entire block with no confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=100 logNum=1 offset=0 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 2 +bytes read: 0 + +# Multiple encoding issues with no confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=100 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=100 logNum=1 offset=0 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Multiple encoding issues but large offset is also has issues, no confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=100 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=100 logNum=1 offset=100000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Simple encoding issues with confirmation +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=100 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=30000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 0 + +# Issues in first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a confirmation chunk. Observe that final blockNum == 1 +init +writeChunk encodedLength=1 chunkLength=1 encoding=100 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 0 + +# Issues in first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a encoding issue chunk, leading to EOF +init +writeChunk encodedLength=1 chunkLength=1 encoding=100 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=100 logNum=1 offset=32000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 0 + +# Complex multiple encoding issues with no confirmation +init +writeChunk encodedLength=1 chunkLength=1 encoding=100 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=100 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=100 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=100 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=100 logNum=1 offset=64000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 5 +bytes read: 0 + +# Complex multiple encoding issues with confirmation in blockNum 3 +init +writeChunk encodedLength=1 chunkLength=1 encoding=100 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=100 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=100 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=32000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 3 +bytes read: 0 + +# Complex multiple encoding issue with confirmation offset too small +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=100 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=100 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=100 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32709 chunkLength=32709 encoding=9 logNum=1 offset=100 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=100 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 5 +bytes read: 32749 + +########################### +## header format too big ## +########################### +# if r.end+headerSize > r.n { + +# large header chunk +init +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=60000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 32738 + +# Simple large header issues in single block with no confirmation +init +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 2 +bytes read: 32738 + +# Multiple large header issues with no confirmation +init +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 32738 + +# Multiple large header issues but large offset is also has issues, no confirmation +init +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=1000000 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 32738 + +# Simple large header issues with confirmation +init +writeChunk encodedLength=32738 chunkLength=32738 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=100000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 32738 + +# Issues in first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a confirmation chunk. Observe that final blockNum == 1 +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +writeChunk encodedLength=32718 chunkLength=32718 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=1000000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 1 +bytes read: 1 + +# Issues in first block with confirming chunks after. However, these next chunks +# in the same block cannot be used to confirm because reading ahead jumps to the next +# block which has a zeroed chunk, leading to EOF +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=60 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +writeChunk encodedLength=32718 chunkLength=32718 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=32738 chunkLength=32738 encoding=0 logNum=1 offset=32000 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 3 +bytes read: 1 + +# Multiple large header issues with confirmation +init +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +writeChunk encodedLength=32718 chunkLength=32718 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=1 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +writeChunk encodedLength=32718 chunkLength=32718 encoding=9 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=64000 corrupt=false +EOF logNum=2 +---- + +read +---- +error reading next: pebble/record: invalid chunk +final blockNum: 2 +bytes read: 1 + +# Complex multiple large header chunks with confirmation offset too small +init +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=0 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=0 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +writeChunk encodedLength=32718 chunkLength=32718 encoding=9 logNum=1 offset=60 corrupt=false +writeChunk encodedLength=1 chunkLength=1 encoding=9 logNum=1 offset=1 corrupt=false +raw 0x01 0x01 0x01 0x01 0x01 0x01 0x09 0x00 0x00 0x00 0x00 # length 11, 0x09 encodes a wal sync type, which is 19 bytes header +writeChunk encodedLength=32718 chunkLength=32718 encoding=9 logNum=1 offset=32000 corrupt=false +writeChunk encodedLength=32749 chunkLength=32749 encoding=9 logNum=1 offset=100 corrupt=false + +EOF logNum=2 +---- + +read +---- +error reading next: unexpected EOF +final blockNum: 5 +bytes read: 32750 \ No newline at end of file diff --git a/testdata/checkpoint b/testdata/checkpoint index 9140cec42d..7a5a409f3b 100644 --- a/testdata/checkpoint +++ b/testdata/checkpoint @@ -43,6 +43,10 @@ create: db/marker.format-version.000006.019 close: db/marker.format-version.000006.019 remove: db/marker.format-version.000005.018 sync: db +create: db/marker.format-version.000007.020 +close: db/marker.format-version.000007.020 +remove: db/marker.format-version.000006.019 +sync: db create: db/temporary.000003.dbtmp sync: db/temporary.000003.dbtmp close: db/temporary.000003.dbtmp @@ -109,9 +113,9 @@ sync-data: checkpoints/checkpoint1/OPTIONS-000003 close: checkpoints/checkpoint1/OPTIONS-000003 close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint1 -create: checkpoints/checkpoint1/marker.format-version.000001.019 -sync-data: checkpoints/checkpoint1/marker.format-version.000001.019 -close: checkpoints/checkpoint1/marker.format-version.000001.019 +create: checkpoints/checkpoint1/marker.format-version.000001.020 +sync-data: checkpoints/checkpoint1/marker.format-version.000001.020 +close: checkpoints/checkpoint1/marker.format-version.000001.020 sync: checkpoints/checkpoint1 close: checkpoints/checkpoint1 link: db/000005.sst -> checkpoints/checkpoint1/000005.sst @@ -153,9 +157,9 @@ sync-data: checkpoints/checkpoint2/OPTIONS-000003 close: checkpoints/checkpoint2/OPTIONS-000003 close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint2 -create: checkpoints/checkpoint2/marker.format-version.000001.019 -sync-data: checkpoints/checkpoint2/marker.format-version.000001.019 -close: checkpoints/checkpoint2/marker.format-version.000001.019 +create: checkpoints/checkpoint2/marker.format-version.000001.020 +sync-data: checkpoints/checkpoint2/marker.format-version.000001.020 +close: checkpoints/checkpoint2/marker.format-version.000001.020 sync: checkpoints/checkpoint2 close: checkpoints/checkpoint2 link: db/000007.sst -> checkpoints/checkpoint2/000007.sst @@ -192,9 +196,9 @@ sync-data: checkpoints/checkpoint3/OPTIONS-000003 close: checkpoints/checkpoint3/OPTIONS-000003 close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint3 -create: checkpoints/checkpoint3/marker.format-version.000001.019 -sync-data: checkpoints/checkpoint3/marker.format-version.000001.019 -close: checkpoints/checkpoint3/marker.format-version.000001.019 +create: checkpoints/checkpoint3/marker.format-version.000001.020 +sync-data: checkpoints/checkpoint3/marker.format-version.000001.020 +close: checkpoints/checkpoint3/marker.format-version.000001.020 sync: checkpoints/checkpoint3 close: checkpoints/checkpoint3 link: db/000005.sst -> checkpoints/checkpoint3/000005.sst @@ -278,7 +282,7 @@ list db LOCK MANIFEST-000001 OPTIONS-000003 -marker.format-version.000006.019 +marker.format-version.000007.020 marker.manifest.000001.MANIFEST-000001 list checkpoints/checkpoint1 @@ -288,7 +292,7 @@ list checkpoints/checkpoint1 000007.sst MANIFEST-000001 OPTIONS-000003 -marker.format-version.000001.019 +marker.format-version.000001.020 marker.manifest.000001.MANIFEST-000001 open checkpoints/checkpoint1 readonly @@ -355,7 +359,7 @@ list checkpoints/checkpoint2 000007.sst MANIFEST-000001 OPTIONS-000003 -marker.format-version.000001.019 +marker.format-version.000001.020 marker.manifest.000001.MANIFEST-000001 open checkpoints/checkpoint2 readonly @@ -397,7 +401,7 @@ list checkpoints/checkpoint3 000007.sst MANIFEST-000001 OPTIONS-000003 -marker.format-version.000001.019 +marker.format-version.000001.020 marker.manifest.000001.MANIFEST-000001 open checkpoints/checkpoint3 readonly @@ -516,9 +520,9 @@ sync-data: checkpoints/checkpoint4/OPTIONS-000003 close: checkpoints/checkpoint4/OPTIONS-000003 close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint4 -create: checkpoints/checkpoint4/marker.format-version.000001.019 -sync-data: checkpoints/checkpoint4/marker.format-version.000001.019 -close: checkpoints/checkpoint4/marker.format-version.000001.019 +create: checkpoints/checkpoint4/marker.format-version.000001.020 +sync-data: checkpoints/checkpoint4/marker.format-version.000001.020 +close: checkpoints/checkpoint4/marker.format-version.000001.020 sync: checkpoints/checkpoint4 close: checkpoints/checkpoint4 link: db/000010.sst -> checkpoints/checkpoint4/000010.sst @@ -606,7 +610,7 @@ list db LOCK MANIFEST-000001 OPTIONS-000003 -marker.format-version.000006.019 +marker.format-version.000007.020 marker.manifest.000001.MANIFEST-000001 @@ -625,9 +629,9 @@ sync-data: checkpoints/checkpoint5/OPTIONS-000003 close: checkpoints/checkpoint5/OPTIONS-000003 close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint5 -create: checkpoints/checkpoint5/marker.format-version.000001.019 -sync-data: checkpoints/checkpoint5/marker.format-version.000001.019 -close: checkpoints/checkpoint5/marker.format-version.000001.019 +create: checkpoints/checkpoint5/marker.format-version.000001.020 +sync-data: checkpoints/checkpoint5/marker.format-version.000001.020 +close: checkpoints/checkpoint5/marker.format-version.000001.020 sync: checkpoints/checkpoint5 close: checkpoints/checkpoint5 link: db/000010.sst -> checkpoints/checkpoint5/000010.sst @@ -727,9 +731,9 @@ sync-data: checkpoints/checkpoint6/OPTIONS-000003 close: checkpoints/checkpoint6/OPTIONS-000003 close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint6 -create: checkpoints/checkpoint6/marker.format-version.000001.019 -sync-data: checkpoints/checkpoint6/marker.format-version.000001.019 -close: checkpoints/checkpoint6/marker.format-version.000001.019 +create: checkpoints/checkpoint6/marker.format-version.000001.020 +sync-data: checkpoints/checkpoint6/marker.format-version.000001.020 +close: checkpoints/checkpoint6/marker.format-version.000001.020 sync: checkpoints/checkpoint6 close: checkpoints/checkpoint6 link: db/000011.sst -> checkpoints/checkpoint6/000011.sst diff --git a/testdata/checkpoint_shared b/testdata/checkpoint_shared index 9836137967..a5bdb82cfb 100644 --- a/testdata/checkpoint_shared +++ b/testdata/checkpoint_shared @@ -31,6 +31,10 @@ create: db/marker.format-version.000003.019 close: db/marker.format-version.000003.019 remove: db/marker.format-version.000002.018 sync: db +create: db/marker.format-version.000004.020 +close: db/marker.format-version.000004.020 +remove: db/marker.format-version.000003.019 +sync: db create: db/temporary.000003.dbtmp sync: db/temporary.000003.dbtmp close: db/temporary.000003.dbtmp @@ -97,9 +101,9 @@ sync-data: checkpoints/checkpoint1/OPTIONS-000003 close: checkpoints/checkpoint1/OPTIONS-000003 close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint1 -create: checkpoints/checkpoint1/marker.format-version.000001.019 -sync-data: checkpoints/checkpoint1/marker.format-version.000001.019 -close: checkpoints/checkpoint1/marker.format-version.000001.019 +create: checkpoints/checkpoint1/marker.format-version.000001.020 +sync-data: checkpoints/checkpoint1/marker.format-version.000001.020 +close: checkpoints/checkpoint1/marker.format-version.000001.020 sync: checkpoints/checkpoint1 close: checkpoints/checkpoint1 open: db/MANIFEST-000001 (options: *vfs.sequentialReadsOption) @@ -150,9 +154,9 @@ sync-data: checkpoints/checkpoint2/OPTIONS-000003 close: checkpoints/checkpoint2/OPTIONS-000003 close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint2 -create: checkpoints/checkpoint2/marker.format-version.000001.019 -sync-data: checkpoints/checkpoint2/marker.format-version.000001.019 -close: checkpoints/checkpoint2/marker.format-version.000001.019 +create: checkpoints/checkpoint2/marker.format-version.000001.020 +sync-data: checkpoints/checkpoint2/marker.format-version.000001.020 +close: checkpoints/checkpoint2/marker.format-version.000001.020 sync: checkpoints/checkpoint2 close: checkpoints/checkpoint2 open: db/MANIFEST-000001 (options: *vfs.sequentialReadsOption) @@ -199,9 +203,9 @@ sync-data: checkpoints/checkpoint3/OPTIONS-000003 close: checkpoints/checkpoint3/OPTIONS-000003 close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint3 -create: checkpoints/checkpoint3/marker.format-version.000001.019 -sync-data: checkpoints/checkpoint3/marker.format-version.000001.019 -close: checkpoints/checkpoint3/marker.format-version.000001.019 +create: checkpoints/checkpoint3/marker.format-version.000001.020 +sync-data: checkpoints/checkpoint3/marker.format-version.000001.020 +close: checkpoints/checkpoint3/marker.format-version.000001.020 sync: checkpoints/checkpoint3 close: checkpoints/checkpoint3 open: db/MANIFEST-000001 (options: *vfs.sequentialReadsOption) @@ -258,7 +262,7 @@ LOCK MANIFEST-000001 OPTIONS-000003 REMOTE-OBJ-CATALOG-000001 -marker.format-version.000003.019 +marker.format-version.000004.020 marker.manifest.000001.MANIFEST-000001 marker.remote-obj-catalog.000001.REMOTE-OBJ-CATALOG-000001 @@ -268,7 +272,7 @@ list checkpoints/checkpoint1 MANIFEST-000001 OPTIONS-000003 REMOTE-OBJ-CATALOG-000001 -marker.format-version.000001.019 +marker.format-version.000001.020 marker.manifest.000001.MANIFEST-000001 marker.remote-obj-catalog.000001.REMOTE-OBJ-CATALOG-000001 @@ -320,7 +324,7 @@ list checkpoints/checkpoint2 MANIFEST-000001 OPTIONS-000003 REMOTE-OBJ-CATALOG-000001 -marker.format-version.000001.019 +marker.format-version.000001.020 marker.manifest.000001.MANIFEST-000001 marker.remote-obj-catalog.000001.REMOTE-OBJ-CATALOG-000001 diff --git a/testdata/event_listener b/testdata/event_listener index 6e99c722ae..928e07fc5e 100644 --- a/testdata/event_listener +++ b/testdata/event_listener @@ -56,6 +56,11 @@ close: db/marker.format-version.000006.019 remove: db/marker.format-version.000005.018 sync: db upgraded to format version: 019 +create: db/marker.format-version.000007.020 +close: db/marker.format-version.000007.020 +remove: db/marker.format-version.000006.019 +sync: db +upgraded to format version: 020 create: db/temporary.000003.dbtmp sync: db/temporary.000003.dbtmp close: db/temporary.000003.dbtmp @@ -365,9 +370,9 @@ sync-data: checkpoint/OPTIONS-000003 close: checkpoint/OPTIONS-000003 close: db/OPTIONS-000003 open-dir: checkpoint -create: checkpoint/marker.format-version.000001.019 -sync-data: checkpoint/marker.format-version.000001.019 -close: checkpoint/marker.format-version.000001.019 +create: checkpoint/marker.format-version.000001.020 +sync-data: checkpoint/marker.format-version.000001.020 +close: checkpoint/marker.format-version.000001.020 sync: checkpoint close: checkpoint link: db/000013.sst -> checkpoint/000013.sst diff --git a/testdata/flushable_ingest b/testdata/flushable_ingest index fa78d2948d..36112ec7e8 100644 --- a/testdata/flushable_ingest +++ b/testdata/flushable_ingest @@ -60,7 +60,7 @@ LOCK MANIFEST-000001 OPTIONS-000003 ext -marker.format-version.000006.019 +marker.format-version.000007.020 marker.manifest.000001.MANIFEST-000001 # Test basic WAL replay @@ -81,7 +81,7 @@ LOCK MANIFEST-000001 OPTIONS-000003 ext -marker.format-version.000006.019 +marker.format-version.000007.020 marker.manifest.000001.MANIFEST-000001 open @@ -390,7 +390,7 @@ LOCK MANIFEST-000001 OPTIONS-000003 ext -marker.format-version.000006.019 +marker.format-version.000007.020 marker.manifest.000001.MANIFEST-000001 close @@ -410,7 +410,7 @@ LOCK MANIFEST-000001 OPTIONS-000003 ext -marker.format-version.000006.019 +marker.format-version.000007.020 marker.manifest.000001.MANIFEST-000001 open @@ -443,7 +443,7 @@ MANIFEST-000001 MANIFEST-000011 OPTIONS-000014 ext -marker.format-version.000006.019 +marker.format-version.000007.020 marker.manifest.000002.MANIFEST-000011 # Make sure that the new mutable memtable can accept writes. @@ -586,7 +586,7 @@ LOCK MANIFEST-000001 OPTIONS-000003 ext -marker.format-version.000006.019 +marker.format-version.000007.020 marker.manifest.000001.MANIFEST-000001 close @@ -605,7 +605,7 @@ MANIFEST-000001 OPTIONS-000003 ext ext1 -marker.format-version.000006.019 +marker.format-version.000007.020 marker.manifest.000001.MANIFEST-000001 open diff --git a/tool/testdata/db_upgrade b/tool/testdata/db_upgrade index 7c27d73c8f..5a94e6ce8c 100644 --- a/tool/testdata/db_upgrade +++ b/tool/testdata/db_upgrade @@ -27,7 +27,7 @@ db get foo yellow db upgrade foo ---- ---- -Upgrading DB from internal version 16 to 19. +Upgrading DB from internal version 16 to 20. WARNING!!! This DB will not be usable with older versions of Pebble! @@ -43,7 +43,7 @@ Continue? [Y/N] Error: EOF db upgrade foo --yes ---- -Upgrading DB from internal version 16 to 19. +Upgrading DB from internal version 16 to 20. Upgrade complete. db get foo blue @@ -56,4 +56,4 @@ db get foo yellow db upgrade foo ---- -DB is already at internal version 19. +DB is already at internal version 20. diff --git a/wal/failover_manager.go b/wal/failover_manager.go index 7acaf2a4db..6814bcb45f 100644 --- a/wal/failover_manager.go +++ b/wal/failover_manager.go @@ -631,6 +631,7 @@ func (wm *failoverManager) Create(wn NumWAL, jobID int) (Writer, error) { failoverWriteAndSyncLatency: wm.opts.FailoverWriteAndSyncLatency, writerClosed: wm.writerClosed, writerCreatedForTest: wm.opts.logWriterCreatedForTesting, + writeWALSyncOffsets: wm.opts.WriteWALSyncOffsets, } var err error var ww *failoverWriter diff --git a/wal/failover_writer.go b/wal/failover_writer.go index 198e4bd83e..0b73e3a537 100644 --- a/wal/failover_writer.go +++ b/wal/failover_writer.go @@ -463,6 +463,9 @@ type failoverWriterOpts struct { writerClosed func(logicalLogWithSizesEtc) writerCreatedForTest chan<- struct{} + + // writeWALSyncOffsets represents whether to write the WAL sync chunk format. + writeWALSyncOffsets bool } func simpleLogCreator( @@ -633,6 +636,7 @@ func (ww *failoverWriter) switchToNewDir(dir dirAndFileHandle) error { WALFsyncLatency: ww.opts.fsyncLatency, QueueSemChan: ww.opts.queueSemChan, ExternalSyncQueueCallback: ww.doneSyncCallback, + WriteWALSyncOffsets: ww.opts.writeWALSyncOffsets, }) closeWriter := func() bool { ww.mu.Lock() diff --git a/wal/standalone_manager.go b/wal/standalone_manager.go index 2c22836953..26f707416e 100644 --- a/wal/standalone_manager.go +++ b/wal/standalone_manager.go @@ -194,9 +194,10 @@ func (m *StandaloneManager) Create(wn NumWAL, jobID int) (Writer, error) { PreallocateSize: m.o.PreallocateSize(), }) w := record.NewLogWriter(newLogFile, newLogNum, record.LogWriterConfig{ - WALFsyncLatency: m.o.FsyncLatency, - WALMinSyncInterval: m.o.MinSyncInterval, - QueueSemChan: m.o.QueueSemChan, + WALFsyncLatency: m.o.FsyncLatency, + WALMinSyncInterval: m.o.MinSyncInterval, + QueueSemChan: m.o.QueueSemChan, + WriteWALSyncOffsets: m.o.WriteWALSyncOffsets, }) m.w = &standaloneWriter{ m: m, diff --git a/wal/testdata/reader b/wal/testdata/reader index 257bf8c0ad..3256c6acb6 100644 --- a/wal/testdata/reader +++ b/wal/testdata/reader @@ -90,7 +90,7 @@ r.NextRecord() = (rr, (000002.log: 111), ) r.NextRecord() = (rr, (000002.log: 272), ) io.ReadAll(rr) = ("2a0000000000000001000000ec8367c42ebf0ffad5c57ece37b18559ba95ad78... <64000-byte record>", ) BatchHeader: [seqNum=42,count=1] -r.NextRecord() = (rr, (000002.log: 64294), pebble/record: invalid chunk) +r.NextRecord() = (rr, (000002.log: 64294), unexpected EOF) # Test a typical failure scenario. Start off with a recycled log file (000003) # that would be on the primary device. It closes "unclean" because we're unable @@ -253,7 +253,7 @@ r.NextRecord() = (rr, (000005-001.log: 55), 6022 from previous files, ) r.NextRecord() = (rr, (000005-001.log: 482), 6022 from previous files, ) io.ReadAll(rr) = ("12750100000000001d0000007575c6296b096226e5e78b9760aa7c2ecfa913b6... <199-byte record>", ) BatchHeader: [seqNum=95506,count=29] -r.NextRecord() = (rr, (000005-001.log: 692), 6022 from previous files, pebble/record: invalid chunk) +r.NextRecord() = (rr, (000005-001.log: 692), 6022 from previous files, unexpected EOF) # Read again, this time pretending we found a third segment with the # logNameIndex=002. This helps exercise error conditions switching to a new diff --git a/wal/wal.go b/wal/wal.go index 2616442c5e..5291d01669 100644 --- a/wal/wal.go +++ b/wal/wal.go @@ -146,6 +146,10 @@ type Options struct { // FailoverWriteAndSyncLatency is only populated when WAL failover is // configured. FailoverWriteAndSyncLatency prometheus.Histogram + + // WriteWALSyncOffsets represents whether to write the WAL sync chunk format. + // It is plumbed down from wal.Options to record.newLogWriter. + WriteWALSyncOffsets bool } // Init constructs and initializes a WAL manager from the provided options and