Skip to content

Commit

Permalink
tool: add remotecat dump command
Browse files Browse the repository at this point in the history
This commit adds a `remotecat dump` command to the pebble tool. This
shows the contents (creator ID and objects) of a remote object
catalog. If the `--verbose` flag is passed, it also shows each record
in the catalog file.

Fixes #2721.
  • Loading branch information
RaduBerinde committed Jul 25, 2023
1 parent 7ef7553 commit 7a4ed3c
Show file tree
Hide file tree
Showing 10 changed files with 289 additions and 25 deletions.
1 change: 1 addition & 0 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,7 @@ generate-test-data:
${GO} run -tags make_incorrect_manifests ./tool/make_incorrect_manifests.go
${GO} run -tags make_test_find_db ./tool/make_test_find_db.go
${GO} run -tags make_test_sstables ./tool/make_test_sstables.go
${GO} run -tags make_test_remotecat ./tool/make_test_remotecat.go

mod-update:
${GO} get -u
Expand Down
28 changes: 10 additions & 18 deletions objstorage/objstorageprovider/remoteobjcat/catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import (
// Catalog is used to manage the on-disk remote object catalog.
//
// The catalog file is a log of records, where each record is an encoded
// versionEdit.
// VersionEdit.
type Catalog struct {
fs vfs.FS
dirname string
Expand Down Expand Up @@ -136,7 +136,7 @@ func (c *Catalog) SetCreatorID(id objstorage.CreatorID) error {
return nil
}

ve := versionEdit{CreatorID: id}
ve := VersionEdit{CreatorID: id}
if err := c.writeToCatalogFileLocked(&ve); err != nil {
return errors.Wrapf(err, "pebble: could not write to remote object catalog: %v", err)
}
Expand Down Expand Up @@ -165,7 +165,7 @@ func (c *Catalog) closeCatalogFile() error {

// Batch is used to perform multiple object additions/deletions at once.
type Batch struct {
ve versionEdit
ve VersionEdit
}

// AddObject adds a new object to the batch.
Expand Down Expand Up @@ -272,29 +272,21 @@ func (c *Catalog) loadFromCatalogFile(filename string) error {
return errors.Wrapf(err, "pebble: error when loading remote object catalog file %q",
errors.Safe(filename))
}
var ve versionEdit
var ve VersionEdit
err = ve.Decode(r)
if err != nil {
return errors.Wrapf(err, "pebble: error when loading remote object catalog file %q",
errors.Safe(filename))
}
// Apply the version edit to the current state.
if ve.CreatorID.IsSet() {
c.mu.creatorID = ve.CreatorID
}
for _, fileNum := range ve.DeletedObjects {
delete(c.mu.objects, fileNum)
}
for _, meta := range ve.NewObjects {
c.mu.objects[meta.FileNum] = meta
}
ve.Apply(&c.mu.creatorID, c.mu.objects)
}
return nil
}

// writeToCatalogFileLocked writes a versionEdit to the catalog file.
// writeToCatalogFileLocked writes a VersionEdit to the catalog file.
// Creates a new file if this is the first write.
func (c *Catalog) writeToCatalogFileLocked(ve *versionEdit) error {
func (c *Catalog) writeToCatalogFileLocked(ve *VersionEdit) error {
c.mu.rotationHelper.AddRecord(int64(len(ve.NewObjects) + len(ve.DeletedObjects)))
snapshotSize := int64(len(c.mu.objects))

Expand Down Expand Up @@ -337,8 +329,8 @@ func (c *Catalog) createNewCatalogFileLocked() (outErr error) {
}
recWriter := record.NewWriter(file)
err = func() error {
// Create a versionEdit that gets us from an empty catalog to the current state.
var ve versionEdit
// Create a VersionEdit that gets us from an empty catalog to the current state.
var ve VersionEdit
ve.CreatorID = c.mu.creatorID
ve.NewObjects = make([]RemoteObjectMetadata, 0, len(c.mu.objects))
for _, meta := range c.mu.objects {
Expand Down Expand Up @@ -375,7 +367,7 @@ func (c *Catalog) createNewCatalogFileLocked() (outErr error) {
return nil
}

func writeRecord(ve *versionEdit, file vfs.File, recWriter *record.Writer) error {
func writeRecord(ve *VersionEdit, file vfs.File, recWriter *record.Writer) error {
w, err := recWriter.Next()
if err != nil {
return err
Expand Down
23 changes: 19 additions & 4 deletions objstorage/objstorageprovider/remoteobjcat/version_edit.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,11 @@ import (
"github.com/cockroachdb/pebble/objstorage/remote"
)

// versionEdit is a modification to the remote object state which can be encoded
// VersionEdit is a modification to the remote object state which can be encoded
// into a record.
//
// TODO(radu): consider adding creation and deletion time for debugging purposes.
type versionEdit struct {
type VersionEdit struct {
NewObjects []RemoteObjectMetadata
DeletedObjects []base.DiskFileNum
CreatorID objstorage.CreatorID
Expand Down Expand Up @@ -69,7 +69,7 @@ func fileTypeToObjType(fileType base.FileType) (uint64, error) {
}

// Encode encodes an edit to the specified writer.
func (v *versionEdit) Encode(w io.Writer) error {
func (v *VersionEdit) Encode(w io.Writer) error {
buf := make([]byte, 0, binary.MaxVarintLen64*(len(v.NewObjects)*10+len(v.DeletedObjects)*2+2))
for _, meta := range v.NewObjects {
objType, err := fileTypeToObjType(meta.FileType)
Expand Down Expand Up @@ -107,7 +107,7 @@ func (v *versionEdit) Encode(w io.Writer) error {
}

// Decode decodes an edit from the specified reader.
func (v *versionEdit) Decode(r io.Reader) error {
func (v *VersionEdit) Decode(r io.Reader) error {
br, ok := r.(io.ByteReader)
if !ok {
br = bufio.NewReader(r)
Expand Down Expand Up @@ -225,3 +225,18 @@ func decodeString(br io.ByteReader) (string, error) {
}

var errCorruptCatalog = base.CorruptionErrorf("pebble: corrupt remote object catalog")

// Apply the version edit to a creator ID and a map of objects.
func (v *VersionEdit) Apply(
creatorID *objstorage.CreatorID, objects map[base.DiskFileNum]RemoteObjectMetadata,
) {
if v.CreatorID.IsSet() {
*creatorID = v.CreatorID
}
for _, fileNum := range v.DeletedObjects {
delete(objects, fileNum)
}
for _, meta := range v.NewObjects {
objects[meta.FileNum] = meta
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ import (
)

func TestVersionEditRoundTrip(t *testing.T) {
for _, ve := range []versionEdit{
for _, ve := range []VersionEdit{
{},
{
CreatorID: 12345,
Expand Down Expand Up @@ -76,8 +76,8 @@ func TestVersionEditRoundTrip(t *testing.T) {
}
}

func checkRoundTrip(e0 versionEdit) error {
var e1 versionEdit
func checkRoundTrip(e0 VersionEdit) error {
var e1 VersionEdit
buf := new(bytes.Buffer)
if err := e0.Encode(buf); err != nil {
return errors.Wrap(err, "encode")
Expand Down
77 changes: 77 additions & 0 deletions tool/make_test_remotecat.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
// Copyright 2023 The LevelDB-Go and Pebble Authors. All rights reserved. Use
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.

//go:build make_test_remotecat
// +build make_test_remotecat

// Run using: go run -tags make_test_remotecat ./tool/make_test_remotecat.go
package main

import (
"log"
"os"
"path/filepath"

"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/objstorage"
"github.com/cockroachdb/pebble/objstorage/objstorageprovider/remoteobjcat"
"github.com/cockroachdb/pebble/vfs"
)

func main() {
dir, err := os.MkdirTemp("", "make-test-remotecat")
if err != nil {
log.Fatal(err)
}
catalog, _, err := remoteobjcat.Open(vfs.Default, dir)
if err != nil {
log.Fatal(err)
}
if err := catalog.SetCreatorID(3); err != nil {
log.Fatal(err)
}

var b remoteobjcat.Batch
b.AddObject(remoteobjcat.RemoteObjectMetadata{
FileNum: base.FileNum(1).DiskFileNum(),
FileType: base.FileTypeTable,
CreatorID: 3,
CreatorFileNum: base.FileNum(1).DiskFileNum(),
CleanupMethod: objstorage.SharedRefTracking,
Locator: "foo",
})
if err := catalog.ApplyBatch(b); err != nil {
log.Fatal(err)
}
b.Reset()
b.AddObject(remoteobjcat.RemoteObjectMetadata{
FileNum: base.FileNum(2).DiskFileNum(),
FileType: base.FileTypeTable,
CreatorID: 5,
CreatorFileNum: base.FileNum(10).DiskFileNum(),
CleanupMethod: objstorage.SharedRefTracking,
Locator: "foo",
})
b.DeleteObject(base.FileNum(1).DiskFileNum())
b.AddObject(remoteobjcat.RemoteObjectMetadata{
FileNum: base.FileNum(3).DiskFileNum(),
FileType: base.FileTypeTable,
CleanupMethod: objstorage.SharedRefTracking,
Locator: "bar",
CustomObjectName: "external.sst",
})
if err := catalog.ApplyBatch(b); err != nil {
log.Fatal(err)
}
if err := catalog.Close(); err != nil {
log.Fatal(err)
}
contents, err := os.ReadFile(filepath.Join(dir, "REMOTE-OBJ-CATALOG-000001"))
if err != nil {
log.Fatal(err)
}
if err := os.WriteFile("tool/testdata/REMOTE-OBJ-CATALOG", contents, 0666); err != nil {
log.Fatal(err)
}
}
132 changes: 132 additions & 0 deletions tool/remotecat.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,132 @@
// Copyright 2023 The LevelDB-Go and Pebble Authors. All rights reserved. Use
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.

package tool

import (
"fmt"
"io"
"sort"

"github.com/cockroachdb/pebble"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/objstorage"
"github.com/cockroachdb/pebble/objstorage/objstorageprovider/remoteobjcat"
"github.com/cockroachdb/pebble/record"
"github.com/spf13/cobra"
)

// remoteCatalogT implements tools for the remote object catalog.
type remoteCatalogT struct {
Root *cobra.Command
Dump *cobra.Command

verbose bool
opts *pebble.Options
}

func newRemoteCatalog(opts *pebble.Options) *remoteCatalogT {
m := &remoteCatalogT{
opts: opts,
}

m.Root = &cobra.Command{
Use: "remotecat",
Short: "remote object catalog introspection tools",
}

// Add dump command
m.Dump = &cobra.Command{
Use: "dump <remote-catalog-files>",
Short: "print remote object catalog contents",
Long: `
Print the contents of the REMOTE-OBJ-CATALOG files.
`,
Args: cobra.MinimumNArgs(1),
Run: m.runDump,
}
m.Dump.Flags().BoolVarP(&m.verbose, "verbose", "v", false, "show each record in the catalog")
m.Root.AddCommand(m.Dump)

return m
}

func (m *remoteCatalogT) runDump(cmd *cobra.Command, args []string) {
for _, arg := range args {
err := m.runDumpOne(cmd.OutOrStdout(), arg)
if err != nil {
fmt.Fprintf(cmd.OutOrStderr(), "%s\n", err)
}
}
}

func (m *remoteCatalogT) runDumpOne(stdout io.Writer, filename string) error {
f, err := m.opts.FS.Open(filename)
if err != nil {
return err
}

var creatorID objstorage.CreatorID
objects := make(map[base.DiskFileNum]remoteobjcat.RemoteObjectMetadata)

fmt.Fprintf(stdout, "%s\n", filename)
var editIdx int
rr := record.NewReader(f, 0 /* logNum */)
for {
offset := rr.Offset()
r, err := rr.Next()
if err == io.EOF {
break
} else if err != nil {
return err
}

var ve remoteobjcat.VersionEdit
err = ve.Decode(r)
if err != nil {
return err
}

if m.verbose {
fmt.Fprintf(stdout, "%d/%d\n", offset, editIdx)
if ve.CreatorID.IsSet() {
fmt.Fprintf(stdout, " CreatorID: %s\n", ve.CreatorID)
}
if len(ve.NewObjects) > 0 {
fmt.Fprintf(stdout, " NewObjects:\n")
for _, m := range ve.NewObjects {
fmt.Fprintf(
stdout, " %s CreatorID: %s CreatorFileNum: %s Locator: %q CustomObjectName: %q\n",
m.FileNum, m.CreatorID, m.CreatorFileNum, m.Locator, m.CustomObjectName,
)
}
}
if len(ve.DeletedObjects) > 0 {
fmt.Fprintf(stdout, " DeletedObjects:\n")
for _, n := range ve.DeletedObjects {
fmt.Fprintf(stdout, " %s\n", n)
}
}
}
editIdx++
ve.Apply(&creatorID, objects)
}
fmt.Fprintf(stdout, "CreatorID: %v\n", creatorID)
var filenums []base.DiskFileNum
for n := range objects {
filenums = append(filenums, n)
}
sort.Slice(filenums, func(i, j int) bool {
return filenums[i].FileNum() < filenums[j].FileNum()
})
fmt.Fprintf(stdout, "Objects:\n")
for _, n := range filenums {
m := objects[n]
fmt.Fprintf(
stdout, " %s CreatorID: %s CreatorFileNum: %s Locator: %q CustomObjectName: %q\n",
n, m.CreatorID, m.CreatorFileNum, m.Locator, m.CustomObjectName,
)
}
return nil
}
11 changes: 11 additions & 0 deletions tool/remotecat_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,11 @@
// Copyright 2023 The LevelDB-Go and Pebble Authors. All rights reserved. Use
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.

package tool

import "testing"

func TestRemotecat(t *testing.T) {
runTests(t, "testdata/remotecat")
}
Binary file added tool/testdata/REMOTE-OBJ-CATALOG
Binary file not shown.
Loading

0 comments on commit 7a4ed3c

Please sign in to comment.