From 66ffff9afdcc8030dd094e30b387790c3076c6d3 Mon Sep 17 00:00:00 2001 From: Paul Banks Date: Tue, 28 Mar 2023 14:15:12 +0100 Subject: [PATCH 1/3] Add dump to filer for debugging --- segment/filer.go | 146 ++++++++++++++++++++++++++++++++-- segment/filer_test.go | 181 ++++++++++++++++++++++++++++++++++++++++++ segment/writer.go | 153 +++++++++++++++++++---------------- 3 files changed, 407 insertions(+), 73 deletions(-) diff --git a/segment/filer.go b/segment/filer.go index fdff616..ae7b6b3 100644 --- a/segment/filer.go +++ b/segment/filer.go @@ -120,13 +120,18 @@ func (f *Filer) Open(info types.SegmentInfo) (types.SegmentReader, error) { // unclean shutdown. The returned map is a map of ID -> BaseIndex. BaseIndex // is returned to allow subsequent Delete calls to be made. func (f *Filer) List() (map[uint64]uint64, error) { - segs := make(map[uint64]uint64) + segs, _, err := f.listInternal() + return segs, err +} +func (f *Filer) listInternal() (map[uint64]uint64, []uint64, error) { files, err := f.vfs.ListDir(f.dir) if err != nil { - return nil, err + return nil, nil, err } + segs := make(map[uint64]uint64) + sorted := make([]uint64, 0) for _, file := range files { if !strings.HasSuffix(file, segmentFileSuffix) { continue @@ -135,17 +140,18 @@ func (f *Filer) List() (map[uint64]uint64, error) { var bIdx, id uint64 n, err := fmt.Sscanf(file, segmentFileNamePattern, &bIdx, &id) if err != nil { - return nil, types.ErrCorrupt + return nil, nil, types.ErrCorrupt } if n != 2 { // Misnamed segment files with the right suffix indicates a bug or // tampering, we can't be sure what's happened to the data. - return nil, types.ErrCorrupt + return nil, nil, types.ErrCorrupt } segs[id] = bIdx + sorted = append(sorted, id) } - return segs, nil + return segs, sorted, nil } // Delete removes the segment with given baseIndex and id if it exists. Note @@ -157,3 +163,133 @@ func (f *Filer) Delete(baseIndex uint64, ID uint64) error { fname := fmt.Sprintf(segmentFileNamePattern, baseIndex, ID) return f.vfs.Delete(f.dir, fname) } + +// DumpSegment attempts to read the segment file specified by the baseIndex and +// ID. It's intended purpose is for debugging the contents of segment files and +// unlike the SegmentFiler interface, it doesn't assume the caller has access to +// the correct metadata. This allows dumping log segments in a WAL that is still +// being written to by another process. Without metadata we don't know if the +// file is sealed so always recover by reading through the whole file. If after +// or before are non-zero, the specify a exclusive lower or upper bound on which +// log entries should be emitted. No error checking is done on the read data. fn +// is called for each entry passing the raft info read from the file header (so +// that the caller knows which codec to use for example) the raft index of the +// entry and the raw bytes of the entry itself. The callback must return true to +// continue reading. The data slice is only valid for the lifetime of the call. +func (f *Filer) DumpSegment(baseIndex uint64, ID uint64, after, before uint64, fn func(info types.SegmentInfo, e types.LogEntry) (bool, error)) error { + fname := fmt.Sprintf(segmentFileNamePattern, baseIndex, ID) + + rf, err := f.vfs.OpenReader(f.dir, fname) + if err != nil { + return err + } + + buf := make([]byte, 64*1024) + idx := baseIndex + + type frameInfo struct { + Index uint64 + Offset int64 + Len uint32 + } + var batch []frameInfo + + _, err = readThroughSegment(rf, func(info types.SegmentInfo, fh frameHeader, offset int64) (bool, error) { + if fh.typ == FrameCommit { + // All the previous entries have been committed. Read them and send up to + // caller. + for _, frame := range batch { + // Check the header is reasonable + if frame.Len > MaxEntrySize { + return false, fmt.Errorf("failed to read entry idx=%d, frame header length (%d) is too big: %w", + frame.Index, frame.Len, err) + } + + if frame.Len > uint32(len(buf)) { + buf = make([]byte, frame.Len) + } + + n, err := rf.ReadAt(buf[:frame.Len], frame.Offset+frameHeaderLen) + if err != nil { + return false, err + } + if uint32(n) < frame.Len { + return false, io.ErrUnexpectedEOF + } + + ok, err := fn(info, types.LogEntry{Index: frame.Index, Data: buf[:n]}) + if !ok || err != nil { + return ok, err + } + } + // Reset batch + batch = batch[:0] + return true, nil + } + + if fh.typ != FrameEntry { + return true, nil + } + + if idx <= after { + // Not in the range we care about, skip reading the entry. + idx++ + return true, nil + } + if before > 0 && idx >= before { + // We're done + return false, nil + } + + batch = append(batch, frameInfo{idx, offset, fh.len}) + idx++ + return true, nil + }) + + return err +} + +// DumpLogs attempts to read all log entries from segment files in the directory +// for debugging purposes. It does _not_ use the metadata and so may output log +// entries that are uncommitted or already truncated as far as the writing +// process is concerned. As such it should not be used for replication of data. +// It is useful though to debug the contents of the log even while the writing +// application is still running. After and before if non-zero specify exclusive +// bounds on the logs that should be returned which may allow the implementation +// to skip reading entire segment files that are not in the range. +func (f *Filer) DumpLogs(after, before uint64, fn func(info types.SegmentInfo, e types.LogEntry) (bool, error)) error { + baseIndexes, segIDsSorted, err := f.listInternal() + if err != nil { + return err + } + + for i, id := range segIDsSorted { + baseIndex := baseIndexes[id] + nextBaseIndex := uint64(0) + if i+1 < len(segIDsSorted) { + // This is not the last segment, peek at the base index of that one and + // assume that this segment won't contain indexes that high. + nextBaseIndex = baseIndexes[segIDsSorted[i+1]] + } + // See if this file contains any indexes in the range + if after > 0 && nextBaseIndex > 0 && after >= nextBaseIndex { + // This segment is all indexes before the lower bound we care about + continue + } + if before > 0 && before <= baseIndex { + // This segment is all indexes higher than the upper bound. We've output + // every log in the range at this point (barring edge cases where we race + // with a truncation which leaves multiple generations of segment files on + // disk which we are going to ignore for now). + return nil + } + + // We probably care about at least some of the entries in this segment + err := f.DumpSegment(baseIndex, id, after, before, fn) + if err != nil { + return err + } + } + + return nil +} diff --git a/segment/filer_test.go b/segment/filer_test.go index d110fc8..8ec03e4 100644 --- a/segment/filer_test.go +++ b/segment/filer_test.go @@ -520,3 +520,184 @@ func testSegment(baseIndex uint64) types.SegmentInfo { // Other fields don't really matter at segment level for now. } } + +func TestDumpSegment(t *testing.T) { + vfs := newTestVFS() + + f := NewFiler("test", vfs) + + // Create one sealed segments + idx := uint64(1) + seg1 := testSegment(idx) + w, err := f.Create(seg1) + require.NoError(t, err) + + var sealed bool + for sealed == false { + val := fmt.Sprintf("%05d. Some Value.", idx) + err = w.Append([]types.LogEntry{{Index: idx, Data: []byte(val)}}) + require.NoError(t, err) + + sealed, _, err = w.Sealed() + require.NoError(t, err) + idx++ + } + w.Close() + + // And one tail + seg2 := testSegment(idx) + w, err = f.Create(seg2) + require.NoError(t, err) + err = w.Append([]types.LogEntry{{Index: idx, Data: []byte("tail")}}) + require.NoError(t, err) + idx++ + defer w.Close() + + // Now dump and make sure we see all the entries + lastDumpedIdx := uint64(0) + totalDumped := 0 + err = f.DumpSegment(seg1.BaseIndex, seg1.ID, 0, 0, func(info types.SegmentInfo, e types.LogEntry) (bool, error) { + require.Equal(t, seg1.BaseIndex, info.BaseIndex) + require.Equal(t, seg1.ID, info.ID) + require.Equal(t, seg1.Codec, info.Codec) + require.Equal(t, lastDumpedIdx+1, e.Index) + require.Equal(t, fmt.Sprintf("%05d. Some Value.", e.Index), string(e.Data)) + totalDumped++ + lastDumpedIdx = e.Index + return true, nil + }) + require.NoError(t, err) + require.Equal(t, 93, totalDumped) + + err = f.DumpSegment(seg2.BaseIndex, seg2.ID, 0, 0, func(info types.SegmentInfo, e types.LogEntry) (bool, error) { + require.Equal(t, seg2.BaseIndex, info.BaseIndex) + require.Equal(t, seg2.ID, info.ID) + require.Equal(t, seg2.Codec, info.Codec) + require.Equal(t, lastDumpedIdx+1, e.Index) + require.Equal(t, "tail", string(e.Data)) + totalDumped++ + lastDumpedIdx = idx + return true, nil + }) + require.NoError(t, err) + require.Equal(t, 94, totalDumped) + + // Ensure if we ask to stop that we stop + totalDumped = 0 + err = f.DumpSegment(seg1.BaseIndex, seg1.ID, 0, 0, func(info types.SegmentInfo, e types.LogEntry) (bool, error) { + totalDumped++ + return false, nil + }) + require.NoError(t, err) + require.Equal(t, 1, totalDumped) + + // Ensure if we error it is passed back + totalDumped = 0 + err = f.DumpSegment(seg1.BaseIndex, seg1.ID, 0, 0, func(info types.SegmentInfo, e types.LogEntry) (bool, error) { + totalDumped++ + return true, fmt.Errorf("bad") + }) + require.ErrorContains(t, err, "bad") + require.Equal(t, 1, totalDumped) + + // Ensure reading from a tail with uncommitted appends doesn't return those + + // First we write a new batch to the tail (because the last batch inluded the + // file header). The entry is 8 bytes long to keep padding simple. + err = w.Append([]types.LogEntry{{Index: idx, Data: []byte("12345678")}}) + require.NoError(t, err) + + // Now we twiddle the underlying VFS to zero out the commit frame. + // lastSyncState will point to the offset just before the new record and the + // commit frame will be just after. + file := testFileFor(t, w) + _, err = file.WriteAt(bytes.Repeat([]byte{0}, 1024), int64(file.lastSyncStart+encodedFrameSize(8))) + require.NoError(t, err) + + // Now dumping should only return one entry + totalDumped = 0 + err = f.DumpSegment(seg2.BaseIndex, seg2.ID, 0, 0, func(info types.SegmentInfo, e types.LogEntry) (bool, error) { + require.Equal(t, "tail", string(e.Data)) + totalDumped++ + return true, nil + }) + require.NoError(t, err) + require.Equal(t, 1, totalDumped) +} + +func TestDumpLogs(t *testing.T) { + vfs := newTestVFS() + + f := NewFiler("test", vfs) + + // Create three sealed segments + idx := uint64(1) + expectSegments := make([]types.SegmentInfo, 0, 3) + for i := 0; i < 3; i++ { + seg := testSegment(idx) + w, err := f.Create(seg) + require.NoError(t, err) + expectSegments = append(expectSegments, seg) + + var sealed bool + for sealed == false { + val := fmt.Sprintf("%05d. Some Value.", idx) + err = w.Append([]types.LogEntry{{Index: idx, Data: []byte(val)}}) + require.NoError(t, err) + + sealed, _, err = w.Sealed() + require.NoError(t, err) + idx++ + } + w.Close() + } + + // And one tail + seg := testSegment(idx) + w, err := f.Create(seg) + require.NoError(t, err) + expectSegments = append(expectSegments, seg) + val := fmt.Sprintf("%05d. Some Value.", idx) + err = w.Append([]types.LogEntry{{Index: idx, Data: []byte(val)}}) + require.NoError(t, err) + w.Close() + + // Dump everything + totalDumped := 0 + lastDumpedIndex := uint64(0) + lastSegID := -1 + segIndex := -1 + + verifyFn := func(info types.SegmentInfo, e types.LogEntry) (bool, error) { + require.Equal(t, lastDumpedIndex+1, e.Index) + if info.ID != uint64(lastSegID) { + // This is a new segment, move to the next info + segIndex++ + } + expectInfo := expectSegments[segIndex] + require.Equal(t, expectInfo.BaseIndex, info.BaseIndex) + require.Equal(t, expectInfo.ID, info.ID) + require.Equal(t, expectInfo.Codec, info.Codec) + + require.Equal(t, fmt.Sprintf("%05d. Some Value.", e.Index), string(e.Data)) + + lastDumpedIndex = e.Index + lastSegID = int(info.ID) + totalDumped++ + return true, nil + } + + err = f.DumpLogs(0, 0, verifyFn) + require.NoError(t, err) + require.Equal(t, int(idx), totalDumped) + + // Test limiting the range (the code above appends 280 records currently) + totalDumped = 0 + lastDumpedIndex = 150 // We are dumping _after_ 150 + lastSegID = -1 + segIndex = 0 // 151 is in the second segment (index 1) so start from the index before + + err = f.DumpLogs(150, 250, verifyFn) + require.NoError(t, err) + require.Equal(t, int(250-150-1), totalDumped) +} diff --git a/segment/writer.go b/segment/writer.go index c5aa809..c5a7bd5 100644 --- a/segment/writer.go +++ b/segment/writer.go @@ -118,42 +118,6 @@ func (w *Writer) initEmpty() error { } func (w *Writer) recoverTail() error { - // First read the file header. Note we wrote it as part of the first commit so - // it may be missing or partial written and that's OK as long as there are no - // other later commit frames! - var fh [fileHeaderLen]byte - _, err := w.wf.ReadAt(fh[:], 0) - // EOF is ok - the file might be empty if we crashed before committing - // anything and preallocation isn't supported. - if err != io.EOF && err != nil { - return err - } - - readInfo, err := readFileHeader(fh[:]) - if err == types.ErrCorrupt { - // Header is malformed or missing, don't error yet though we'll detect it - // later when we know if it's a problem or not. - err = nil - } - if err != nil { - return err - } - // If header wasn't detected as corrupt, it might still be just in a way - // that's valid since we've not verified it against the expected metadata yet. - // We'll wait to see if the header was part of the last commit before decide - // if we should validate it for corruption or not though. For now just make - // sure it's not nil so we don't have to handle nil checks everywhere. - if readInfo == nil { - // Zero info will fail validation against the actual metadata if it was - // corrupt when it shouldn't be later. Just prevents a nil panic. - readInfo = &types.SegmentInfo{} - } - - // Read through file from after header until we hit zeros, EOF or corrupt - // frames. - offset := int64(fileHeaderLen) - var buf [frameHeaderLen]byte - // We need to track the last two commit frames type commitInfo struct { fh frameHeader @@ -165,36 +129,8 @@ func (w *Writer) recoverTail() error { offsets := make([]uint32, 0, 32*1024) -READ: - for { - n, err := w.wf.ReadAt(buf[:], offset) - if err == io.EOF { - if n < frameHeaderLen { - break READ - } - // This is OK! The last frame in file might be a commit frame so as long - // as we have it all then we can ignore the EOF for this iteration. - err = nil - } - if err != nil { - return fmt.Errorf("failed reading frame at offset=%d: %w", offset, err) - } - fh, err := readFrameHeader(buf[:frameHeaderLen]) - if err != nil { - // This is not actually an error case. If we failed to decode it could be - // because of a torn write (since we don't assume writes are atomic). We - // assume that previously committed data is not silently corrupted by the - // FS (see README for details). So this must be due to corruption that - // happened due to non-atomic sector updates whilst committing the last - // write batch. - break READ - } + readInfo, err := readThroughSegment(w.wf, func(_ types.SegmentInfo, fh frameHeader, offset int64) (bool, error) { switch fh.typ { - case FrameInvalid: - // This means we've hit zeros at the end of the file (or due to an - // incomplete write, which we treat the same way). - break READ - case FrameEntry: // Record the frame offset offsets = append(offsets, uint32(offset)) @@ -218,9 +154,10 @@ READ: finalCommit.crcStart = prevCommit.offset + frameHeaderLen } } - - // Skip to next frame - offset += int64(encodedFrameSize(int(fh.len))) + return true, nil + }) + if err != nil { + return err } if finalCommit == nil { @@ -536,3 +473,83 @@ func (w *Writer) Sealed() (bool, uint64, error) { func (w *Writer) LastIndex() uint64 { return atomic.LoadUint64(&w.commitIdx) } + +func readThroughSegment(r types.ReadableFile, fn func(info types.SegmentInfo, fh frameHeader, offset int64) (bool, error)) (*types.SegmentInfo, error) { + // First read the file header. Note we wrote it as part of the first commit so + // it may be missing or partial written and that's OK as long as there are no + // other later commit frames! + var fh [fileHeaderLen]byte + _, err := r.ReadAt(fh[:], 0) + // EOF is ok - the file might be empty if we crashed before committing + // anything and preallocation isn't supported. + if err != io.EOF && err != nil { + return nil, err + } + + readInfo, err := readFileHeader(fh[:]) + if err == types.ErrCorrupt { + // Header is malformed or missing, don't error yet though we'll detect it + // later when we know if it's a problem or not. + err = nil + } + if err != nil { + return nil, err + } + // If header wasn't detected as corrupt, it might still be just in a way + // that's valid since we've not verified it against the expected metadata yet. + // We'll wait to see if the header was part of the last commit before decide + // if we should validate it for corruption or not though. For now just make + // sure it's not nil so we don't have to handle nil checks everywhere. + if readInfo == nil { + // Zero info will fail validation against the actual metadata if it was + // corrupt when it shouldn't be later. Just prevents a nil panic. + readInfo = &types.SegmentInfo{} + } + + // Read through file from after header until we hit zeros, EOF or corrupt + // frames. + offset := int64(fileHeaderLen) + var buf [frameHeaderLen]byte + + for { + n, err := r.ReadAt(buf[:], offset) + if err == io.EOF { + if n < frameHeaderLen { + return readInfo, nil + } + // This is OK! The last frame in file might be a commit frame so as long + // as we have it all then we can ignore the EOF for this iteration. + err = nil + } + if err != nil { + return readInfo, fmt.Errorf("failed reading frame at offset=%d: %w", offset, err) + } + fh, err := readFrameHeader(buf[:frameHeaderLen]) + if err != nil { + // This is not actually an error case. If we failed to decode it could be + // because of a torn write (since we don't assume writes are atomic). We + // assume that previously committed data is not silently corrupted by the + // FS (see README for details). So this must be due to corruption that + // happened due to non-atomic sector updates whilst committing the last + // write batch. + return readInfo, nil + } + if fh.typ == FrameInvalid { + // This means we've hit zeros at the end of the file (or due to an + // incomplete write, which we treat the same way). + return readInfo, nil + } + + // Call the callback + shouldContinue, err := fn(*readInfo, fh, offset) + if err != nil { + return readInfo, err + } + if !shouldContinue { + return readInfo, nil + } + + // Skip to next frame + offset += int64(encodedFrameSize(int(fh.len))) + } +} From 9f1b7ae906cc9d74645e648aba49bc7b97f02e97 Mon Sep 17 00:00:00 2001 From: Paul Banks Date: Tue, 28 Mar 2023 14:15:29 +0100 Subject: [PATCH 2/3] Add simple dump command --- cmd/waldump/.gitignore | 1 + cmd/waldump/README.md | 36 ++++++++++++++++++++++++ cmd/waldump/waldump.go | 62 ++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 99 insertions(+) create mode 100644 cmd/waldump/.gitignore create mode 100644 cmd/waldump/README.md create mode 100644 cmd/waldump/waldump.go diff --git a/cmd/waldump/.gitignore b/cmd/waldump/.gitignore new file mode 100644 index 0000000..0ec38ec --- /dev/null +++ b/cmd/waldump/.gitignore @@ -0,0 +1 @@ +waldump \ No newline at end of file diff --git a/cmd/waldump/README.md b/cmd/waldump/README.md new file mode 100644 index 0000000..34061f3 --- /dev/null +++ b/cmd/waldump/README.md @@ -0,0 +1,36 @@ +# waldump + +A simple command for dumping the contents of WAL segment files to JSON for +debugging. + +## Usage + +``` +$ waldump /path/to/wal/dir [-after INDEX] [-before INDEX] +... +{"Index":227281,"Term":4,"Type":0,"Data":"hpGEpUNvb3JkhKpBZGp1c3RtZW50yz7pEPrkTc4tpUVycm9yyz/B4NJg87MZpkhlaWdodMs/ABkEWHeDZqNWZWOYyz8FyF63P/XOyz8Fe2fyqYpayz7eXgvdsOWVyz7xX/ARy9MByz7XZq0fmx5eyz7x8ic7zxhJy78EgvusSgKUy77xVfw2sEr5pE5vZGWiczGpUGFydGl0aW9uoKdTZWdtZW50oA==","Extensions":null,"AppendedAt":"2023-03-23T12:24:05.440317Z"} +... +``` + +Each `raft.Log` is written out as JSON followed by a newline. The `Data` and +`Extensions` fields are opaque byte strings that will be base64 encoded. +Decoding those requires knowledge of the encoding used by the writing +application. + +## Limitations + +This tool is designed for debugging only. It does _not_ inspect the wal-meta +database. This has the nice property that you can safely dump the contexts of +WAL files even while the application is still writing to the WAL since we don't +have to take a lock on the meta database. + +The downside is that this tool might in some edge cases output logs that have +already been deleted from the WAL. It's possible although extremely unlikely +that the WAL could be in the process of truncating the tail which could result +in there being both pre-truncate and post-truncate segment files present. This +tool might possibly output duplicate and out-of-order log indexes from before +and after the truncation. Or if `before` and `after` are used, it's possible we +might skip records entirely because an older file that has already been removed +was read instead of the newer one. These are all very unlikely in practice and +if the application that writes the WAL is still up and running are likely to be +resolved by the time you run the tool again. \ No newline at end of file diff --git a/cmd/waldump/waldump.go b/cmd/waldump/waldump.go new file mode 100644 index 0000000..3a31e1e --- /dev/null +++ b/cmd/waldump/waldump.go @@ -0,0 +1,62 @@ +// Copyright (c) HashiCorp, Inc. + +package main + +import ( + "encoding/json" + "flag" + "fmt" + "os" + + "github.com/hashicorp/raft" + wal "github.com/hashicorp/raft-wal" + "github.com/hashicorp/raft-wal/fs" + "github.com/hashicorp/raft-wal/segment" + "github.com/hashicorp/raft-wal/types" +) + +type opts struct { + Dir string + After uint64 + Before uint64 +} + +func main() { + var o opts + flag.Uint64Var(&o.After, "after", 0, "specified a raft index to use as an exclusive lower bound when dumping log entries.") + flag.Uint64Var(&o.Before, "before", 0, "specified a raft index to use as an exclusive upper bound when dumping log entries.") + + flag.Parse() + + // Accept dir as positional arg + o.Dir = flag.Arg(0) + if o.Dir == "" { + fmt.Println("Usage: waldump [-after INDEX] [-before INDEX]") + os.Exit(1) + } + + vfs := fs.New() + f := segment.NewFiler(o.Dir, vfs) + + codec := &wal.BinaryCodec{} + var log raft.Log + enc := json.NewEncoder(os.Stdout) + + err := f.DumpLogs(o.After, o.Before, func(info types.SegmentInfo, e types.LogEntry) (bool, error) { + if info.Codec != wal.CodecBinaryV1 { + return false, fmt.Errorf("unsupported codec %d in file %s", info.Codec, segment.FileName(info)) + } + if err := codec.Decode(e.Data, &log); err != nil { + return false, err + } + // Output the raft Log struct as JSON + if err := enc.Encode(log); err != nil { + return false, err + } + return true, nil + }) + if err != nil { + fmt.Printf("ERROR: %s\n", err) + os.Exit(1) + } +} From 8b98e5fe31b22c334684689a3c3e7da7e2be1069 Mon Sep 17 00:00:00 2001 From: Paul Banks Date: Thu, 30 Mar 2023 14:08:45 +0100 Subject: [PATCH 3/3] Fix waldump usage --- cmd/waldump/README.md | 2 +- cmd/waldump/waldump.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cmd/waldump/README.md b/cmd/waldump/README.md index 34061f3..1d93d51 100644 --- a/cmd/waldump/README.md +++ b/cmd/waldump/README.md @@ -6,7 +6,7 @@ debugging. ## Usage ``` -$ waldump /path/to/wal/dir [-after INDEX] [-before INDEX] +$ waldump [-after INDEX] [-before INDEX] /path/to/wal/dir ... {"Index":227281,"Term":4,"Type":0,"Data":"hpGEpUNvb3JkhKpBZGp1c3RtZW50yz7pEPrkTc4tpUVycm9yyz/B4NJg87MZpkhlaWdodMs/ABkEWHeDZqNWZWOYyz8FyF63P/XOyz8Fe2fyqYpayz7eXgvdsOWVyz7xX/ARy9MByz7XZq0fmx5eyz7x8ic7zxhJy78EgvusSgKUy77xVfw2sEr5pE5vZGWiczGpUGFydGl0aW9uoKdTZWdtZW50oA==","Extensions":null,"AppendedAt":"2023-03-23T12:24:05.440317Z"} ... diff --git a/cmd/waldump/waldump.go b/cmd/waldump/waldump.go index 3a31e1e..82a2786 100644 --- a/cmd/waldump/waldump.go +++ b/cmd/waldump/waldump.go @@ -31,7 +31,7 @@ func main() { // Accept dir as positional arg o.Dir = flag.Arg(0) if o.Dir == "" { - fmt.Println("Usage: waldump [-after INDEX] [-before INDEX]") + fmt.Println("Usage: waldump [-after INDEX] [-before INDEX] ") os.Exit(1) }