├── .circleci └── config.yml ├── LICENSE-APACHE ├── LICENSE-MIT ├── README.md ├── accessor.go ├── accessor_test.go ├── dagstore.go ├── dagstore_async.go ├── dagstore_control.go ├── dagstore_gc.go ├── dagstore_test.go ├── dispatcher.go ├── docs └── design.md ├── gen └── main.go ├── go.mod ├── go.sum ├── handlers.go ├── index ├── inverted_index_impl.go ├── inverted_index_impl_test.go ├── inverted_index_interface.go ├── manifest.go ├── repo.go ├── repo_fs.go ├── repo_fs_test.go ├── repo_mem.go ├── repo_mem_test.go └── repo_test.go ├── indexbs ├── indexbacked_bs.go └── indexbacked_bs_test.go ├── interface.go ├── mount ├── bytes.go ├── counting.go ├── file.go ├── file_test.go ├── fs.go ├── mount.go ├── registry.go ├── registry_test.go ├── upgrader.go └── upgrader_test.go ├── shard.go ├── shard ├── key.go └── key_test.go ├── shard_persist.go ├── shard_state.go ├── shard_state_test.go ├── testdata ├── files │ ├── junk.dat │ ├── sample-v1.car │ └── sample-wrapped-v2.car └── testdata.go └── throttle ├── doc.go ├── throttler.go └── throttler_test.go /.circleci/config.yml: -------------------------------------------------------------------------------- 1 | version: 2.1 2 | orbs: 3 | go: circleci/go@1.7.1 4 | workflows: 5 | main: 6 | jobs: 7 | - build 8 | jobs: 9 | build: 10 | executor: 11 | name: go/default 12 | tag: '1.18' 13 | steps: 14 | - checkout 15 | - go/load-cache 16 | - go/mod-download 17 | - go/save-cache 18 | - go/test: 19 | covermode: atomic 20 | failfast: true 21 | race: true 22 | verbose: true -------------------------------------------------------------------------------- /LICENSE-APACHE: -------------------------------------------------------------------------------- 1 | Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at 2 | 3 | http://www.apache.org/licenses/LICENSE-2.0 4 | 5 | Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. 6 | -------------------------------------------------------------------------------- /LICENSE-MIT: -------------------------------------------------------------------------------- 1 | The MIT License (MIT) 2 | 3 | Permission is hereby granted, free of charge, to any person obtaining a copy 4 | of this software and associated documentation files (the "Software"), to deal 5 | in the Software without restriction, including without limitation the rights 6 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | copies of the Software, and to permit persons to whom the Software is 8 | furnished to do so, subject to the following conditions: 9 | 10 | The above copyright notice and this permission notice shall be included in 11 | all copies or substantial portions of the Software. 12 | 13 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | THE SOFTWARE. 20 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | ## DAG store 2 | 3 | This README will be populated soon. In the meantime, please refer to the 4 | [design document](https://github.com/filecoin-project/dagstore/blob/master/docs/design.md). 5 | 6 | ## License 7 | 8 | Dual-licensed: [MIT](./LICENSE-MIT), [Apache Software License v2](./LICENSE-APACHE), by way of the 9 | [Permissive License Stack](https://protocol.ai/blog/announcing-the-permissive-license-stack/). 10 | -------------------------------------------------------------------------------- /accessor.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | import ( 4 | "context" 5 | "io" 6 | "os" 7 | "sync" 8 | 9 | "github.com/filecoin-project/dagstore/mount" 10 | "github.com/filecoin-project/dagstore/shard" 11 | 12 | blocks "github.com/ipfs/go-block-format" 13 | "github.com/ipfs/go-cid" 14 | carv2 "github.com/ipld/go-car/v2" 15 | "github.com/ipld/go-car/v2/blockstore" 16 | "github.com/ipld/go-car/v2/index" 17 | 18 | "golang.org/x/exp/mmap" 19 | ) 20 | 21 | // ReadBlockstore is a read-only view of Blockstores. This will be implemented 22 | // by the CARv2 indexed blockstore. 23 | type ReadBlockstore interface { 24 | Has(context.Context, cid.Cid) (bool, error) 25 | Get(context.Context, cid.Cid) (blocks.Block, error) 26 | GetSize(context.Context, cid.Cid) (int, error) 27 | AllKeysChan(ctx context.Context) (<-chan cid.Cid, error) 28 | HashOnRead(enabled bool) 29 | } 30 | 31 | // ShardAccessor provides various means to access the data contained 32 | // in a shard. 33 | type ShardAccessor struct { 34 | data mount.Reader 35 | idx index.Index 36 | shard *Shard 37 | 38 | // mmapr is an optional mmap.ReaderAt. It will be non-nil if the mount 39 | // has been mmapped because the mount.Reader was an underlying *os.File, 40 | // and an mmap-backed accessor was requested (e.g. Blockstore). 41 | lk sync.Mutex 42 | mmapr *mmap.ReaderAt 43 | } 44 | 45 | func NewShardAccessor(data mount.Reader, idx index.Index, s *Shard) (*ShardAccessor, error) { 46 | return &ShardAccessor{ 47 | data: data, 48 | idx: idx, 49 | shard: s, 50 | }, nil 51 | } 52 | 53 | func (sa *ShardAccessor) Shard() shard.Key { 54 | return sa.shard.key 55 | } 56 | 57 | func (sa *ShardAccessor) Blockstore() (ReadBlockstore, error) { 58 | var r io.ReaderAt = sa.data 59 | 60 | sa.lk.Lock() 61 | if f, ok := sa.data.(*os.File); ok { 62 | if mmapr, err := mmap.Open(f.Name()); err != nil { 63 | log.Warnf("failed to mmap reader of type %T: %s; using reader as-is", sa.data, err) 64 | } else { 65 | // we don't close the mount.Reader file descriptor because the user 66 | // may have called other non-mmap-backed accessors. 67 | r = mmapr 68 | sa.mmapr = mmapr 69 | } 70 | } 71 | sa.lk.Unlock() 72 | 73 | bs, err := blockstore.NewReadOnly(r, sa.idx, carv2.ZeroLengthSectionAsEOF(true)) 74 | return bs, err 75 | } 76 | 77 | // Close terminates this shard accessor, releasing any resources associated 78 | // with it, and decrementing internal refcounts. 79 | func (sa *ShardAccessor) Close() error { 80 | if err := sa.data.Close(); err != nil { 81 | log.Warnf("failed to close mount when closing shard accessor: %s", err) 82 | } 83 | sa.lk.Lock() 84 | if sa.mmapr != nil { 85 | if err := sa.mmapr.Close(); err != nil { 86 | log.Warnf("failed to close mmap when closing shard accessor: %s", err) 87 | } 88 | } 89 | sa.lk.Unlock() 90 | 91 | tsk := &task{op: OpShardRelease, shard: sa.shard} 92 | return sa.shard.d.queueTask(tsk, sa.shard.d.externalCh) 93 | } 94 | -------------------------------------------------------------------------------- /accessor_test.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | import ( 4 | "context" 5 | "os" 6 | "os/exec" 7 | "path/filepath" 8 | "runtime" 9 | "strconv" 10 | "strings" 11 | "testing" 12 | 13 | "github.com/filecoin-project/dagstore/mount" 14 | "github.com/filecoin-project/dagstore/testdata" 15 | "github.com/filecoin-project/dagstore/throttle" 16 | "github.com/ipld/go-car/v2" 17 | "github.com/ipld/go-car/v2/index" 18 | "github.com/stretchr/testify/require" 19 | ) 20 | 21 | // TestMmap works on linux and darwin. It tests that for the given mount, if 22 | // multiple accessors are opened, the corresponding file (specified by name) 23 | // will be mmapped or not (depending on expect). 24 | // 25 | // It opens multiple ShardAccessors with Blockstores, so as to create as several 26 | // mmaps. It then closes one accessor, and ensures that the other two are still 27 | // operational (i.e. the shard hasn't been mmapped). Once all accessors are 28 | // closed, it verifies that the shard has been munmap'ped. 29 | // 30 | // Several mount types are tested for their corresponding behaviour. Only mounts 31 | // capable of returning *os.File as a mount.Reader will end up being mmapped. 32 | // Both the FileMount and the Upgrader fall in that category. 33 | func TestMmap(t *testing.T) { 34 | run := func(t *testing.T, mnt mount.Mount, expect bool, name string) { 35 | ctx := context.TODO() 36 | // create three shard accessors, with a blockstore each. 37 | var accessors []*ShardAccessor 38 | accessors = append(accessors, createAccessor(t, mnt)) 39 | accessors = append(accessors, createAccessor(t, mnt)) 40 | accessors = append(accessors, createAccessor(t, mnt)) 41 | 42 | var bss []ReadBlockstore 43 | for i := 0; i < 3; i++ { 44 | bs, err := accessors[i].Blockstore() 45 | require.NoError(t, err) 46 | require.NotNil(t, bs) 47 | bss = append(bss, bs) 48 | } 49 | 50 | // works. 51 | blk, err := bss[0].Get(ctx, testdata.RootCID) 52 | require.NoError(t, err) 53 | require.NotNil(t, blk) 54 | 55 | checkMmapped(t, expect, name) 56 | 57 | // after we close the first accessor, other blockstores are still accessible. 58 | err = accessors[0].Close() 59 | require.NoError(t, err) 60 | 61 | // works. 62 | blk, err = bss[1].Get(ctx, testdata.RootCID) 63 | require.NoError(t, err) 64 | require.NotNil(t, blk) 65 | 66 | // works. 67 | blk, err = bss[2].Get(ctx, testdata.RootCID) 68 | require.NoError(t, err) 69 | require.NotNil(t, blk) 70 | 71 | // still the same mmapped status. 72 | checkMmapped(t, expect, name) 73 | 74 | // close the other accessors. 75 | err = accessors[1].Close() 76 | require.NoError(t, err) 77 | 78 | err = accessors[2].Close() 79 | require.NoError(t, err) 80 | 81 | // not mmapped any longer, in no case 82 | checkMmapped(t, false, name) 83 | } 84 | 85 | // macOS vmmap shows /Users/USER/*/sample-wrapped-v2.car instead of the 86 | // full path, probably hidden for privacy reasons. So let's match only 87 | // by filename. 88 | carv2name := filepath.Base(testdata.RootPathCarV2) 89 | 90 | t.Run("bytes-nommap", func(t *testing.T) { 91 | // BytesMount doesn't return an *os.File, therefore it's not mmappable. 92 | mnt := &mount.BytesMount{Bytes: testdata.CarV2} 93 | 94 | run(t, mnt, false, carv2name) 95 | }) 96 | 97 | t.Run("file-mmap", func(t *testing.T) { 98 | // FileMount does return an *os.File, therefore it's mmappable. 99 | mnt := &mount.FileMount{Path: testdata.RootPathCarV2} 100 | run(t, mnt, true, carv2name) 101 | }) 102 | 103 | t.Run("upgrader-mmap", func(t *testing.T) { 104 | // An upgraded FS mount will mmap its local transient. 105 | var mnt mount.Mount = &mount.FSMount{FS: testdata.FS, Path: testdata.FSPathCarV2} 106 | tempdir := t.TempDir() 107 | 108 | var err error 109 | mnt, err = mount.Upgrade(mnt, throttle.Noop(), tempdir, "foo", "") 110 | require.NoError(t, err) 111 | 112 | // warm up the upgrader so a transient is created, and we can obtain 113 | // its path to test against the mmap query output. 114 | reader, err := mnt.Fetch(context.Background()) 115 | require.NoError(t, err) 116 | err = reader.Close() 117 | require.NoError(t, err) 118 | 119 | // in this case, the file we expect to see mmapped is the transient. 120 | name := filepath.Base(mnt.(*mount.Upgrader).TransientPath()) 121 | run(t, mnt, true, name) 122 | }) 123 | 124 | } 125 | 126 | // checkMmapped uses platform-specific logic to verify if the expected file 127 | // has been mmapped. 128 | func checkMmapped(t *testing.T, expect bool, name string) { 129 | pid := os.Getpid() 130 | pred := require.False 131 | if expect { 132 | pred = require.True 133 | } 134 | 135 | var ( 136 | out []byte 137 | err error 138 | ) 139 | 140 | // platform-dependent query of process mmaps 141 | switch runtime.GOOS { 142 | case "darwin": 143 | out, err = exec.Command("vmmap", strconv.Itoa(pid)).CombinedOutput() 144 | 145 | case "linux", "openbsd": 146 | out, err = os.ReadFile("/proc/self/maps") 147 | 148 | default: 149 | t.Skip("unsupported platform.") 150 | } 151 | 152 | require.NoError(t, err) 153 | 154 | t.Logf("vmmap or /proc/self/maps output:") 155 | t.Log(string(out)) 156 | 157 | pred(t, strings.Contains(string(out), name)) 158 | } 159 | 160 | func createAccessor(t *testing.T, mnt mount.Mount) *ShardAccessor { 161 | dummyShard := &Shard{ 162 | d: &DAGStore{ 163 | ctx: context.Background(), 164 | externalCh: make(chan *task, 64), 165 | }, 166 | } 167 | 168 | reader, err := mnt.Fetch(context.Background()) 169 | require.NoError(t, err) 170 | defer reader.Close() 171 | 172 | // skip version reading because this API consumes too many bytes. 173 | // v, err := car.ReadVersion(reader) 174 | // require.NoError(t, err) 175 | // require.EqualValues(t, 2, v) 176 | // skip the pragma instead 177 | _, err = reader.Seek(car.PragmaSize, 0) 178 | require.NoError(t, err) 179 | 180 | var h car.Header 181 | _, err = h.ReadFrom(reader) 182 | require.NoError(t, err) 183 | 184 | _, err = reader.Seek(int64(h.IndexOffset), 0) 185 | require.NoError(t, err) 186 | 187 | idx, err := index.ReadFrom(reader) 188 | require.NoError(t, err) 189 | 190 | reader2, err := mnt.Fetch(context.Background()) 191 | require.NoError(t, err) 192 | 193 | accessor, err := NewShardAccessor(reader2, idx, dummyShard) 194 | require.NoError(t, err) 195 | return accessor 196 | } 197 | -------------------------------------------------------------------------------- /dagstore.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | import ( 4 | "context" 5 | "errors" 6 | "fmt" 7 | "os" 8 | "sync" 9 | 10 | mh "github.com/multiformats/go-multihash" 11 | 12 | "github.com/ipld/go-car/v2" 13 | carindex "github.com/ipld/go-car/v2/index" 14 | 15 | ds "github.com/ipfs/go-datastore" 16 | "github.com/ipfs/go-datastore/namespace" 17 | "github.com/ipfs/go-datastore/query" 18 | dssync "github.com/ipfs/go-datastore/sync" 19 | logging "github.com/ipfs/go-log/v2" 20 | 21 | "github.com/filecoin-project/dagstore/index" 22 | "github.com/filecoin-project/dagstore/mount" 23 | "github.com/filecoin-project/dagstore/shard" 24 | "github.com/filecoin-project/dagstore/throttle" 25 | ) 26 | 27 | var ( 28 | // StoreNamespace is the namespace under which shard state will be persisted. 29 | StoreNamespace = ds.NewKey("dagstore") 30 | ) 31 | 32 | // RecoverOnStartPolicy specifies the recovery policy for failed 33 | // shards on DAGStore start. 34 | type RecoverOnStartPolicy int 35 | 36 | const ( 37 | // DoNotRecover will not recover any failed shards on start. Recovery 38 | // must be performed manually. 39 | DoNotRecover RecoverOnStartPolicy = iota 40 | 41 | // RecoverOnAcquire will automatically queue a recovery for a failed shard 42 | // on the first acquire attempt, and will park that acquire while recovery 43 | // is in progress. 44 | RecoverOnAcquire 45 | 46 | // RecoverNow will eagerly trigger a recovery for all failed shards 47 | // upon start. 48 | RecoverNow 49 | ) 50 | 51 | var log = logging.Logger("dagstore") 52 | 53 | var ( 54 | // ErrShardUnknown is the error returned when the requested shard is 55 | // not known to the DAG store. 56 | ErrShardUnknown = errors.New("shard not found") 57 | 58 | // ErrShardExists is the error returned upon registering a duplicate shard. 59 | ErrShardExists = errors.New("shard already exists") 60 | 61 | // ErrShardInitializationFailed is returned when shard initialization fails. 62 | ErrShardInitializationFailed = errors.New("shard initialization failed") 63 | 64 | // ErrShardInUse is returned when the user attempts to destroy a shard that 65 | // is in use. 66 | ErrShardInUse = errors.New("shard in use") 67 | ) 68 | 69 | // DAGStore is the central object of the DAG store. 70 | type DAGStore struct { 71 | lk sync.RWMutex 72 | mounts *mount.Registry 73 | shards map[shard.Key]*Shard 74 | config Config 75 | indices index.FullIndexRepo 76 | store ds.Datastore 77 | indexer ShardIndexer 78 | 79 | // TopLevelIndex is the top level (cid -> []shards) index that maps a cid to all the shards that is present in. 80 | TopLevelIndex index.Inverted 81 | 82 | // Channels owned by us. 83 | // 84 | // externalCh receives external tasks. 85 | externalCh chan *task 86 | // internalCh receives internal tasks to the event loop. 87 | internalCh chan *task 88 | // completionCh receives tasks queued up as a result of async completions. 89 | completionCh chan *task 90 | // dispatchResultsCh is a buffered channel for dispatching results back to 91 | // the application. Serviced by a dispatcher goroutine. 92 | // Note: This pattern decouples the event loop from the application, so a 93 | // failure to consume immediately won't block the event loop. 94 | dispatchResultsCh chan *dispatch 95 | // dispatchFailuresCh is a buffered channel for dispatching shard failures 96 | // back to the application. Serviced by a dispatcher goroutine. 97 | // See note in dispatchResultsCh for background. 98 | dispatchFailuresCh chan *dispatch 99 | // gcCh is where requests for GC are sent. 100 | gcCh chan chan *GCResult 101 | 102 | // Channels not owned by us. 103 | // 104 | // traceCh is where traces on shard operations will be sent, if non-nil. 105 | traceCh chan<- Trace 106 | // failureCh is where shard failures will be notified, if non-nil. 107 | failureCh chan<- ShardResult 108 | 109 | // Throttling. 110 | // 111 | throttleReaadyFetch throttle.Throttler 112 | throttleIndex throttle.Throttler 113 | 114 | // Lifecycle. 115 | // 116 | ctx context.Context 117 | cancelFn context.CancelFunc 118 | wg sync.WaitGroup 119 | } 120 | 121 | var _ Interface = (*DAGStore)(nil) 122 | 123 | type dispatch struct { 124 | w *waiter 125 | res *ShardResult 126 | } 127 | 128 | // Task represents an operation to be performed on a shard or the DAG store. 129 | type task struct { 130 | *waiter 131 | op OpType 132 | shard *Shard 133 | err error 134 | } 135 | 136 | // ShardResult encapsulates a result from an asynchronous operation. 137 | type ShardResult struct { 138 | Key shard.Key 139 | Error error 140 | Accessor *ShardAccessor 141 | } 142 | 143 | type ShardIndexer func(context.Context, shard.Key, mount.Reader) (carindex.Index, error) 144 | 145 | type Config struct { 146 | // TransientsDir is the path to directory where local transient files will 147 | // be created for remote mounts. 148 | TransientsDir string 149 | 150 | // IndexRepo is the full index repo to use. 151 | IndexRepo index.FullIndexRepo 152 | 153 | TopLevelIndex index.Inverted 154 | 155 | // Datastore is the datastore where shard state will be persisted. 156 | Datastore ds.Datastore 157 | 158 | // MountRegistry contains the set of recognized mount types. 159 | MountRegistry *mount.Registry 160 | 161 | // TraceCh is a channel where the caller desires to be notified of every 162 | // shard operation. Publishing to this channel blocks the event loop, so the 163 | // caller must ensure the channel is serviced appropriately. 164 | // 165 | // Note: Not actively consuming from this channel will make the event 166 | // loop block. 167 | TraceCh chan<- Trace 168 | 169 | // FailureCh is a channel to be notified every time that a shard moves to 170 | // ShardStateErrored. A nil value will send no failure notifications. 171 | // Failure events can be used to evaluate the error and call 172 | // DAGStore.RecoverShard if deemed recoverable. 173 | // 174 | // Note: Not actively consuming from this channel will make the event 175 | // loop block. 176 | FailureCh chan<- ShardResult 177 | 178 | // MaxConcurrentIndex is the maximum indexing jobs that can 179 | // run concurrently. 0 (default) disables throttling. 180 | MaxConcurrentIndex int 181 | 182 | // MaxConcurrentReadyFetches is the maximum number of fetches that will 183 | // run concurrently for mounts that are reporting themselves as ready for 184 | // immediate fetch. 0 (default) disables throttling. 185 | MaxConcurrentReadyFetches int 186 | 187 | // RecoverOnStart specifies whether failed shards should be recovered 188 | // on start. 189 | RecoverOnStart RecoverOnStartPolicy 190 | 191 | // ShardIndexer sets a custom callback for determining the index 192 | // mapping of CID->Offset that should be registered for a shard. 193 | ShardIndexer ShardIndexer 194 | } 195 | 196 | // NewDAGStore constructs a new DAG store with the supplied configuration. 197 | // 198 | // You must call Start for processing to begin. 199 | func NewDAGStore(cfg Config) (*DAGStore, error) { 200 | // validate and manage scratch root directory. 201 | if cfg.TransientsDir == "" { 202 | return nil, fmt.Errorf("missing scratch area root path") 203 | } 204 | if err := ensureDir(cfg.TransientsDir); err != nil { 205 | return nil, fmt.Errorf("failed to create scratch root dir: %w", err) 206 | } 207 | 208 | // instantiate the index repo. 209 | if cfg.IndexRepo == nil { 210 | log.Info("using in-memory index store") 211 | cfg.IndexRepo = index.NewMemoryRepo() 212 | } 213 | 214 | if cfg.TopLevelIndex == nil { 215 | log.Info("using in-memory inverted index") 216 | cfg.TopLevelIndex = index.NewInverted(dssync.MutexWrap(ds.NewMapDatastore())) 217 | } 218 | // default indexer 219 | if cfg.ShardIndexer == nil { 220 | cfg.ShardIndexer = func(_ context.Context, _ shard.Key, r mount.Reader) (carindex.Index, error) { 221 | return car.ReadOrGenerateIndex(r, car.ZeroLengthSectionAsEOF(true), car.StoreIdentityCIDs(true)) 222 | } 223 | } 224 | 225 | // handle the datastore. 226 | if cfg.Datastore == nil { 227 | log.Warnf("no datastore provided; falling back to in-mem datastore; shard state will not survive restarts") 228 | cfg.Datastore = dssync.MutexWrap(ds.NewMapDatastore()) // TODO can probably remove mutex wrap, since access is single-threaded 229 | } 230 | 231 | // namespace all store operations. 232 | cfg.Datastore = namespace.Wrap(cfg.Datastore, StoreNamespace) 233 | 234 | if cfg.MountRegistry == nil { 235 | cfg.MountRegistry = mount.NewRegistry() 236 | } 237 | 238 | ctx, cancel := context.WithCancel(context.Background()) 239 | dagst := &DAGStore{ 240 | mounts: cfg.MountRegistry, 241 | config: cfg, 242 | indices: cfg.IndexRepo, 243 | TopLevelIndex: cfg.TopLevelIndex, 244 | shards: make(map[shard.Key]*Shard), 245 | store: cfg.Datastore, 246 | indexer: cfg.ShardIndexer, 247 | externalCh: make(chan *task, 128), // len=128, concurrent external tasks that can be queued up before exercising backpressure. 248 | internalCh: make(chan *task, 1), // len=1, because eventloop will only ever stage another internal event. 249 | completionCh: make(chan *task, 64), // len=64, hitting this limit will just make async tasks wait. 250 | dispatchResultsCh: make(chan *dispatch, 128), // len=128, same as externalCh. 251 | gcCh: make(chan chan *GCResult, 8), 252 | traceCh: cfg.TraceCh, 253 | failureCh: cfg.FailureCh, 254 | throttleIndex: throttle.Noop(), 255 | throttleReaadyFetch: throttle.Noop(), 256 | ctx: ctx, 257 | cancelFn: cancel, 258 | } 259 | 260 | if max := cfg.MaxConcurrentIndex; max > 0 { 261 | dagst.throttleIndex = throttle.Fixed(max) 262 | } 263 | 264 | if max := cfg.MaxConcurrentReadyFetches; max > 0 { 265 | dagst.throttleReaadyFetch = throttle.Fixed(max) 266 | } 267 | 268 | return dagst, nil 269 | } 270 | 271 | // Start starts a DAG store. 272 | func (d *DAGStore) Start(ctx context.Context) error { 273 | if err := d.restoreState(); err != nil { 274 | // TODO add a lenient mode. 275 | return fmt.Errorf("failed to restore dagstore state: %w", err) 276 | } 277 | 278 | if err := d.clearOrphaned(); err != nil { 279 | log.Warnf("failed to clear orphaned files on startup: %s", err) 280 | } 281 | 282 | // Reset in-progress states. 283 | // 284 | // Queue shards whose registration needs to be restarted. Release those 285 | // ops after we spawn the control goroutine. Otherwise, having more shards 286 | // in this state than the externalCh buffer size would exceed the channel 287 | // buffer, and we'd block forever. 288 | var toRegister, toRecover []*Shard 289 | for _, s := range d.shards { 290 | switch s.state { 291 | case ShardStateErrored: 292 | switch d.config.RecoverOnStart { 293 | case DoNotRecover: 294 | log.Infow("start: skipping recovery of shard in errored state", "shard", s.key, "error", s.err) 295 | case RecoverOnAcquire: 296 | log.Infow("start: failed shard will recover on next acquire", "shard", s.key, "error", s.err) 297 | s.recoverOnNextAcquire = true 298 | case RecoverNow: 299 | log.Infow("start: recovering failed shard immediately", "shard", s.key, "error", s.err) 300 | toRecover = append(toRecover, s) 301 | } 302 | 303 | case ShardStateServing: 304 | // reset to available, as we have no active acquirers at start. 305 | s.state = ShardStateAvailable 306 | case ShardStateAvailable: 307 | // Noop: An available shard whose index has disappeared across restarts 308 | // will fail on the first acquisition. 309 | case ShardStateInitializing: 310 | // handle shards that were initializing when we shut down. 311 | // if we already have the index for the shard, there's nothing else to do. 312 | if istat, err := d.indices.StatFullIndex(s.key); err == nil && istat.Exists { 313 | s.state = ShardStateAvailable 314 | } else { 315 | // reset back to new, and queue the OpShardRegister. 316 | s.state = ShardStateNew 317 | toRegister = append(toRegister, s) 318 | } 319 | } 320 | } 321 | 322 | // spawn the control goroutine. 323 | d.wg.Add(1) 324 | go d.control() 325 | 326 | // spawn the dispatcher goroutine for responses, responsible for pumping 327 | // async results back to the caller. 328 | d.wg.Add(1) 329 | go d.dispatcher(d.dispatchResultsCh) 330 | 331 | // application has provided a failure channel; spawn the dispatcher. 332 | if d.failureCh != nil { 333 | d.dispatchFailuresCh = make(chan *dispatch, 128) // len=128, same as externalCh. 334 | d.wg.Add(1) 335 | go d.dispatcher(d.dispatchFailuresCh) 336 | } 337 | 338 | // release the queued registrations before we return. 339 | for _, s := range toRegister { 340 | _ = d.queueTask(&task{op: OpShardRegister, shard: s, waiter: &waiter{ctx: ctx}}, d.externalCh) 341 | } 342 | 343 | // queue shard recovery for shards in the errored state before we return. 344 | for _, s := range toRecover { 345 | _ = d.queueTask(&task{op: OpShardRecover, shard: s, waiter: &waiter{ctx: ctx}}, d.externalCh) 346 | } 347 | 348 | return nil 349 | } 350 | 351 | func (d *DAGStore) GetIterableIndex(key shard.Key) (carindex.IterableIndex, error) { 352 | fi, err := d.indices.GetFullIndex(key) 353 | if err != nil { 354 | return nil, fmt.Errorf("failed to get iterable index: %w", err) 355 | } 356 | 357 | ii, ok := fi.(carindex.IterableIndex) 358 | if !ok { 359 | return nil, errors.New("index for shard is not iterable") 360 | } 361 | 362 | return ii, nil 363 | } 364 | 365 | func (d *DAGStore) ShardsContainingMultihash(ctx context.Context, h mh.Multihash) ([]shard.Key, error) { 366 | return d.TopLevelIndex.GetShardsForMultihash(ctx, h) 367 | } 368 | 369 | type RegisterOpts struct { 370 | // ExistingTransient can be supplied when registering a shard to indicate 371 | // that there's already an existing local transient copy that can be used 372 | // for indexing. 373 | ExistingTransient string 374 | 375 | // LazyInitialization defers shard indexing to the first access instead of 376 | // performing it at registration time. Use this option when fetching the 377 | // asset is expensive. 378 | // 379 | // When true, the registration channel will fire as soon as the DAG store 380 | // has acknowledged the inclusion of the shard, without waiting for any 381 | // indexing to happen. 382 | LazyInitialization bool 383 | } 384 | 385 | // RegisterShard initiates the registration of a new shard. 386 | // 387 | // This method returns an error synchronously if preliminary validation fails. 388 | // Otherwise, it queues the shard for registration. The caller should monitor 389 | // supplied channel for a result. 390 | func (d *DAGStore) RegisterShard(ctx context.Context, key shard.Key, mnt mount.Mount, out chan ShardResult, opts RegisterOpts) error { 391 | d.lk.Lock() 392 | if _, ok := d.shards[key]; ok { 393 | d.lk.Unlock() 394 | return fmt.Errorf("%s: %w", key.String(), ErrShardExists) 395 | } 396 | 397 | // wrap the original mount in an upgrader. 398 | upgraded, err := mount.Upgrade(mnt, d.throttleReaadyFetch, d.config.TransientsDir, key.String(), opts.ExistingTransient) 399 | if err != nil { 400 | d.lk.Unlock() 401 | return err 402 | } 403 | 404 | w := &waiter{outCh: out, ctx: ctx} 405 | 406 | // add the shard to the shard catalogue, and drop the lock. 407 | s := &Shard{ 408 | d: d, 409 | key: key, 410 | state: ShardStateNew, 411 | mount: upgraded, 412 | lazy: opts.LazyInitialization, 413 | } 414 | d.shards[key] = s 415 | d.lk.Unlock() 416 | 417 | tsk := &task{op: OpShardRegister, shard: s, waiter: w} 418 | return d.queueTask(tsk, d.externalCh) 419 | } 420 | 421 | type DestroyOpts struct { 422 | } 423 | 424 | func (d *DAGStore) DestroyShard(ctx context.Context, key shard.Key, out chan ShardResult, _ DestroyOpts) error { 425 | d.lk.Lock() 426 | s, ok := d.shards[key] 427 | if !ok { 428 | d.lk.Unlock() 429 | return ErrShardUnknown // TODO: encode shard key 430 | } 431 | d.lk.Unlock() 432 | 433 | tsk := &task{op: OpShardDestroy, shard: s, waiter: &waiter{ctx: ctx, outCh: out}} 434 | return d.queueTask(tsk, d.externalCh) 435 | } 436 | 437 | type AcquireOpts struct { 438 | } 439 | 440 | // AcquireShard acquires access to the specified shard, and returns a 441 | // ShardAccessor, an object that enables various patterns of access to the data 442 | // contained within the shard. 443 | // 444 | // This operation may resolve near-instantaneously if the shard is available 445 | // locally. If not, the shard data may be fetched from its mount. 446 | // 447 | // This method returns an error synchronously if preliminary validation fails. 448 | // Otherwise, it queues the shard for acquisition. The caller should monitor 449 | // supplied channel for a result. 450 | func (d *DAGStore) AcquireShard(ctx context.Context, key shard.Key, out chan ShardResult, _ AcquireOpts) error { 451 | d.lk.Lock() 452 | s, ok := d.shards[key] 453 | if !ok { 454 | d.lk.Unlock() 455 | return fmt.Errorf("%s: %w", key.String(), ErrShardUnknown) 456 | } 457 | d.lk.Unlock() 458 | 459 | tsk := &task{op: OpShardAcquire, shard: s, waiter: &waiter{ctx: ctx, outCh: out}} 460 | return d.queueTask(tsk, d.externalCh) 461 | } 462 | 463 | type RecoverOpts struct { 464 | } 465 | 466 | // RecoverShard recovers a shard in ShardStateErrored state. 467 | // 468 | // If the shard referenced by the key doesn't exist, an error is returned 469 | // immediately and no result is delivered on the supplied channel. 470 | // 471 | // If the shard is not in the ShardStateErrored state, the operation is accepted 472 | // but an error will be returned quickly on the supplied channel. 473 | // 474 | // Otherwise, the recovery operation will be queued and the supplied channel 475 | // will be notified when it completes. 476 | // 477 | // TODO add an operation identifier to ShardResult -- starts to look like 478 | // 479 | // a Trace event? 480 | func (d *DAGStore) RecoverShard(ctx context.Context, key shard.Key, out chan ShardResult, _ RecoverOpts) error { 481 | d.lk.Lock() 482 | s, ok := d.shards[key] 483 | if !ok { 484 | d.lk.Unlock() 485 | return fmt.Errorf("%s: %w", key.String(), ErrShardUnknown) 486 | } 487 | d.lk.Unlock() 488 | 489 | tsk := &task{op: OpShardRecover, shard: s, waiter: &waiter{ctx: ctx, outCh: out}} 490 | return d.queueTask(tsk, d.externalCh) 491 | } 492 | 493 | type Trace struct { 494 | Key shard.Key 495 | Op OpType 496 | After ShardInfo 497 | } 498 | 499 | type ShardInfo struct { 500 | ShardState 501 | Error error 502 | refs uint32 503 | } 504 | 505 | // GetShardInfo returns the current state of shard with key k. 506 | // 507 | // If the shard is not known, ErrShardUnknown is returned. 508 | func (d *DAGStore) GetShardInfo(k shard.Key) (ShardInfo, error) { 509 | d.lk.RLock() 510 | defer d.lk.RUnlock() 511 | s, ok := d.shards[k] 512 | if !ok { 513 | return ShardInfo{}, ErrShardUnknown 514 | } 515 | 516 | s.lk.RLock() 517 | info := ShardInfo{ShardState: s.state, Error: s.err, refs: s.refs} 518 | s.lk.RUnlock() 519 | return info, nil 520 | } 521 | 522 | type AllShardsInfo map[shard.Key]ShardInfo 523 | 524 | // AllShardsInfo returns the current state of all registered shards, as well as 525 | // any errors. 526 | func (d *DAGStore) AllShardsInfo() AllShardsInfo { 527 | d.lk.RLock() 528 | defer d.lk.RUnlock() 529 | 530 | ret := make(AllShardsInfo, len(d.shards)) 531 | for k, s := range d.shards { 532 | s.lk.RLock() 533 | info := ShardInfo{ShardState: s.state, Error: s.err, refs: s.refs} 534 | s.lk.RUnlock() 535 | ret[k] = info 536 | } 537 | return ret 538 | } 539 | 540 | // GC performs DAG store garbage collection by reclaiming transient files of 541 | // shards that are currently available but inactive, or errored. 542 | // 543 | // GC runs with exclusivity from the event loop. 544 | func (d *DAGStore) GC(ctx context.Context) (*GCResult, error) { 545 | ch := make(chan *GCResult) 546 | select { 547 | case d.gcCh <- ch: 548 | case <-ctx.Done(): 549 | return nil, ctx.Err() 550 | } 551 | 552 | select { 553 | case res := <-ch: 554 | return res, nil 555 | case <-ctx.Done(): 556 | return nil, ctx.Err() 557 | } 558 | } 559 | 560 | func (d *DAGStore) Close() error { 561 | d.cancelFn() 562 | d.wg.Wait() 563 | _ = d.store.Sync(context.TODO(), ds.Key{}) 564 | return nil 565 | } 566 | 567 | func (d *DAGStore) queueTask(tsk *task, ch chan<- *task) error { 568 | select { 569 | case <-d.ctx.Done(): 570 | return fmt.Errorf("dag store closed") 571 | case ch <- tsk: 572 | return nil 573 | } 574 | } 575 | 576 | func (d *DAGStore) restoreState() error { 577 | results, err := d.store.Query(d.ctx, query.Query{}) 578 | if err != nil { 579 | return fmt.Errorf("failed to recover dagstore state from store: %w", err) 580 | } 581 | for { 582 | res, ok := results.NextSync() 583 | if !ok { 584 | return nil 585 | } 586 | s := &Shard{d: d} 587 | if err := s.UnmarshalJSON(res.Value); err != nil { 588 | log.Warnf("failed to recover state of shard %s: %s; skipping", shard.KeyFromString(res.Key), err) 589 | continue 590 | } 591 | 592 | log.Debugw("restored shard state on dagstore startup", "shard", s.key, "shard state", s.state, "shard error", s.err, 593 | "shard lazy", s.lazy) 594 | d.shards[s.key] = s 595 | } 596 | } 597 | 598 | // ensureDir checks whether the specified path is a directory, and if not it 599 | // attempts to create it. 600 | func ensureDir(path string) error { 601 | fi, err := os.Stat(path) 602 | if err != nil { 603 | // We need to create the directory. 604 | return os.MkdirAll(path, os.ModePerm) 605 | } 606 | 607 | if !fi.IsDir() { 608 | return fmt.Errorf("path %s exists, and it is not a directory", path) 609 | } 610 | return nil 611 | } 612 | 613 | // failShard queues a shard failure (does not fail it immediately). It is 614 | // suitable for usage both outside and inside the event loop, depending on the 615 | // channel passed. 616 | func (d *DAGStore) failShard(s *Shard, ch chan *task, format string, args ...interface{}) error { 617 | err := fmt.Errorf(format, args...) 618 | return d.queueTask(&task{op: OpShardFail, shard: s, err: err}, ch) 619 | } 620 | -------------------------------------------------------------------------------- /dagstore_async.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/filecoin-project/dagstore/index" 7 | 8 | carindex "github.com/ipld/go-car/v2/index" 9 | "github.com/multiformats/go-multihash" 10 | 11 | "github.com/filecoin-project/dagstore/mount" 12 | ) 13 | 14 | // 15 | // This file contains methods that are called from the event loop 16 | // but are run asynchronously in dedicated goroutines. 17 | // 18 | 19 | // acquireAsync acquires a shard by fetching its data, obtaining its index, and 20 | // joining them to form a ShardAccessor. 21 | func (d *DAGStore) acquireAsync(ctx context.Context, w *waiter, s *Shard, mnt mount.Mount) { 22 | k := s.key 23 | 24 | reader, err := mnt.Fetch(ctx) 25 | 26 | if err := ctx.Err(); err != nil { 27 | log.Warnw("context cancelled while fetching shard; releasing", "shard", s.key, "error", err) 28 | 29 | // release the shard to decrement the refcount that's incremented before `acquireAsync` is called. 30 | _ = d.queueTask(&task{op: OpShardRelease, shard: s}, d.completionCh) 31 | 32 | // send the shard error to the caller for correctness 33 | // since the context is cancelled, the result will be discarded. 34 | d.dispatchResult(&ShardResult{Key: k, Error: err}, w) 35 | return 36 | } 37 | 38 | if err != nil { 39 | log.Warnw("acquire: failed to fetch from mount upgrader", "shard", s.key, "error", err) 40 | 41 | // release the shard to decrement the refcount that's incremented before `acquireAsync` is called. 42 | _ = d.queueTask(&task{op: OpShardRelease, shard: s}, d.completionCh) 43 | 44 | // fail the shard 45 | _ = d.failShard(s, d.completionCh, "failed to acquire reader of mount so we can return the accessor: %w", err) 46 | 47 | // send the shard error to the caller. 48 | d.dispatchResult(&ShardResult{Key: k, Error: err}, w) 49 | return 50 | } 51 | 52 | log.Debugw("acquire: successfully fetched from mount upgrader", "shard", s.key) 53 | 54 | // acquire the index. 55 | idx, err := d.indices.GetFullIndex(k) 56 | 57 | if err := ctx.Err(); err != nil { 58 | log.Warnw("context cancelled while indexing shard; releasing", "shard", s.key, "error", err) 59 | 60 | // release the shard to decrement the refcount that's incremented before `acquireAsync` is called. 61 | _ = d.queueTask(&task{op: OpShardRelease, shard: s}, d.completionCh) 62 | 63 | // send the shard error to the caller for correctness 64 | // since the context is cancelled, the result will be discarded. 65 | d.dispatchResult(&ShardResult{Key: k, Error: err}, w) 66 | return 67 | } 68 | 69 | if err != nil { 70 | log.Warnw("acquire: failed to get index for shard", "shard", s.key, "error", err) 71 | if err := reader.Close(); err != nil { 72 | log.Errorf("failed to close mount reader: %s", err) 73 | } 74 | 75 | // release the shard to decrement the refcount that's incremented before `acquireAsync` is called. 76 | _ = d.queueTask(&task{op: OpShardRelease, shard: s}, d.completionCh) 77 | 78 | // fail the shard 79 | _ = d.failShard(s, d.completionCh, "failed to recover index for shard %s: %w", k, err) 80 | 81 | // send the shard error to the caller. 82 | d.dispatchResult(&ShardResult{Key: k, Error: err}, w) 83 | return 84 | } 85 | 86 | log.Debugw("acquire: successful; returning accessor", "shard", s.key) 87 | 88 | // build the accessor. 89 | sa, err := NewShardAccessor(reader, idx, s) 90 | 91 | // send the shard accessor to the caller, adding a notifyDead function that 92 | // will be called to release the shard if we were unable to deliver 93 | // the accessor. 94 | w.notifyDead = func() { 95 | log.Warnw("context cancelled while delivering accessor; releasing", "shard", s.key) 96 | 97 | // release the shard to decrement the refcount that's incremented before `acquireAsync` is called. 98 | _ = d.queueTask(&task{op: OpShardRelease, shard: s}, d.completionCh) 99 | } 100 | 101 | d.dispatchResult(&ShardResult{Key: k, Accessor: sa, Error: err}, w) 102 | } 103 | 104 | // initializeShard initializes a shard asynchronously by fetching its data and 105 | // performing indexing. 106 | func (d *DAGStore) initializeShard(ctx context.Context, s *Shard, mnt mount.Mount) { 107 | reader, err := mnt.Fetch(ctx) 108 | if err != nil { 109 | log.Warnw("initialize: failed to fetch from mount upgrader", "shard", s.key, "error", err) 110 | 111 | _ = d.failShard(s, d.completionCh, "failed to acquire reader of mount on initialization: %w", err) 112 | return 113 | } 114 | defer reader.Close() 115 | 116 | log.Debugw("initialize: successfully fetched from mount upgrader", "shard", s.key) 117 | 118 | // works for both CARv1 and CARv2. 119 | var idx carindex.Index 120 | err = d.throttleIndex.Do(ctx, func(_ context.Context) error { 121 | var err error 122 | idx, err = d.indexer(ctx, s.key, reader) 123 | if err == nil { 124 | log.Debugw("initialize: finished generating index for shard", "shard", s.key) 125 | } else { 126 | log.Warnw("initialize: failed to generate index for shard", "shard", s.key, "error", err) 127 | } 128 | return err 129 | }) 130 | if err != nil { 131 | _ = d.failShard(s, d.completionCh, "failed to read/generate CAR Index: %w", err) 132 | return 133 | } 134 | if err := d.indices.AddFullIndex(s.key, idx); err != nil { 135 | _ = d.failShard(s, d.completionCh, "failed to add index for shard: %w", err) 136 | return 137 | } 138 | 139 | // add all cids in the shard to the inverted (cid -> []Shard Keys) index. 140 | iterableIdx, ok := idx.(carindex.IterableIndex) 141 | if ok { 142 | mhIter := &mhIdx{iterableIdx: iterableIdx} 143 | if err := d.TopLevelIndex.AddMultihashesForShard(ctx, mhIter, s.key); err != nil { 144 | log.Errorw("failed to add shard multihashes to the inverted index", "shard", s.key, "error", err) 145 | } 146 | } else { 147 | log.Errorw("shard index is not iterable", "shard", s.key) 148 | } 149 | 150 | _ = d.queueTask(&task{op: OpShardMakeAvailable, shard: s}, d.completionCh) 151 | } 152 | 153 | // Convenience struct for converting from CAR index.IterableIndex to the 154 | // iterator required by the dag store inverted index. 155 | type mhIdx struct { 156 | iterableIdx carindex.IterableIndex 157 | } 158 | 159 | var _ index.MultihashIterator = (*mhIdx)(nil) 160 | 161 | func (it *mhIdx) ForEach(fn func(mh multihash.Multihash) error) error { 162 | return it.iterableIdx.ForEach(func(mh multihash.Multihash, _ uint64) error { 163 | return fn(mh) 164 | }) 165 | } 166 | -------------------------------------------------------------------------------- /dagstore_control.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | 7 | "github.com/ipfs/go-datastore" 8 | ) 9 | 10 | type OpType int 11 | 12 | const ( 13 | OpShardRegister OpType = iota 14 | OpShardInitialize 15 | OpShardMakeAvailable 16 | OpShardDestroy 17 | OpShardAcquire 18 | OpShardFail 19 | OpShardRelease 20 | OpShardRecover 21 | ) 22 | 23 | func (o OpType) String() string { 24 | return [...]string{ 25 | "OpShardRegister", 26 | "OpShardInitialize", 27 | "OpShardMakeAvailable", 28 | "OpShardDestroy", 29 | "OpShardAcquire", 30 | "OpShardFail", 31 | "OpShardRelease", 32 | "OpShardRecover"}[o] 33 | } 34 | 35 | // control runs the DAG store's event loop. 36 | func (d *DAGStore) control() { 37 | defer d.wg.Done() 38 | 39 | // wFailure is a synthetic failure waiter that uses the DAGStore's 40 | // global context and the failure channel. Only safe to actually use if 41 | // d.failureCh != nil. wFailure is used to dispatch failure 42 | // notifications to the application. 43 | var wFailure = &waiter{ctx: d.ctx, outCh: d.failureCh} 44 | 45 | for { 46 | // consume the next task or GC request; if we're shutting down, this method will error. 47 | tsk, gc, err := d.consumeNext() 48 | if err != nil { 49 | if err == context.Canceled { 50 | log.Infow("dagstore closed") 51 | } else { 52 | log.Errorw("consuming next task failed; aborted event loop; dagstore unoperational", "error", err) 53 | } 54 | return 55 | } 56 | 57 | if gc != nil { 58 | // this was a GC request. 59 | d.gc(gc) 60 | continue 61 | } 62 | 63 | s := tsk.shard 64 | log.Debugw("processing task", "op", tsk.op, "shard", tsk.shard.key, "error", tsk.err) 65 | 66 | s.lk.Lock() 67 | prevState := s.state 68 | 69 | switch tsk.op { 70 | case OpShardRegister: 71 | if s.state != ShardStateNew { 72 | // sanity check failed 73 | _ = d.failShard(s, d.internalCh, "%w: expected shard to be in 'new' state; was: %s", ErrShardInitializationFailed, s.state) 74 | break 75 | } 76 | 77 | // skip initialization if shard was registered with lazy init, and 78 | // respond immediately to waiter. 79 | if s.lazy { 80 | log.Debugw("shard registered with lazy initialization", "shard", s.key) 81 | // waiter will be nil if this was a restart and not a call to Register() call. 82 | if tsk.waiter != nil { 83 | res := &ShardResult{Key: s.key} 84 | d.dispatchResult(res, tsk.waiter) 85 | } 86 | break 87 | } 88 | 89 | // otherwise, park the registration channel and queue the init. 90 | s.wRegister = tsk.waiter 91 | _ = d.queueTask(&task{op: OpShardInitialize, shard: s, waiter: tsk.waiter}, d.internalCh) 92 | 93 | case OpShardInitialize: 94 | s.state = ShardStateInitializing 95 | 96 | // if we already have the index for this shard, there's nothing to do here. 97 | if istat, err := d.indices.StatFullIndex(s.key); err == nil && istat.Exists { 98 | log.Debugw("already have an index for shard being initialized, nothing to do", "shard", s.key) 99 | _ = d.queueTask(&task{op: OpShardMakeAvailable, shard: s}, d.internalCh) 100 | break 101 | } 102 | 103 | go d.initializeShard(tsk.ctx, s, s.mount) 104 | 105 | case OpShardMakeAvailable: 106 | // can arrive here after initializing a new shard, 107 | // or when recovering from a failure. 108 | 109 | s.state = ShardStateAvailable 110 | s.err = nil // nillify past errors 111 | 112 | // notify the registration waiter, if there is one. 113 | if s.wRegister != nil { 114 | res := &ShardResult{Key: s.key} 115 | d.dispatchResult(res, s.wRegister) 116 | s.wRegister = nil 117 | } 118 | 119 | // notify the recovery waiter, if there is one. 120 | if s.wRecover != nil { 121 | res := &ShardResult{Key: s.key} 122 | d.dispatchResult(res, s.wRecover) 123 | s.wRecover = nil 124 | } 125 | 126 | // trigger queued acquisition waiters. 127 | for _, w := range s.wAcquire { 128 | s.state = ShardStateServing 129 | 130 | // optimistically increment the refcount to acquire the shard. The go-routine will send an `OpShardRelease` message 131 | // to the event loop if it fails to acquire the shard. 132 | s.refs++ 133 | go d.acquireAsync(w.ctx, w, s, s.mount) 134 | } 135 | s.wAcquire = s.wAcquire[:0] 136 | 137 | case OpShardAcquire: 138 | log.Debugw("got request to acquire shard", "shard", s.key, "current shard state", s.state) 139 | w := &waiter{ctx: tsk.ctx, outCh: tsk.outCh} 140 | 141 | // if the shard is errored, fail the acquire immediately. 142 | if s.state == ShardStateErrored { 143 | if s.recoverOnNextAcquire { 144 | // we are errored, but recovery was requested on the next acquire 145 | // we park the acquirer and trigger a recover. 146 | s.wAcquire = append(s.wAcquire, w) 147 | s.recoverOnNextAcquire = false 148 | // we use the global context instead of the acquire context 149 | // to avoid the first context cancellation interrupting the 150 | // recovery that may be blocking other acquirers with longer 151 | // contexts. 152 | _ = d.queueTask(&task{op: OpShardRecover, shard: s, waiter: &waiter{ctx: d.ctx}}, d.internalCh) 153 | } else { 154 | err := fmt.Errorf("shard is in errored state; err: %w", s.err) 155 | res := &ShardResult{Key: s.key, Error: err} 156 | d.dispatchResult(res, w) 157 | } 158 | break 159 | } 160 | 161 | if s.state != ShardStateAvailable && s.state != ShardStateServing { 162 | log.Debugw("shard isn't active yet, will queue acquire channel", "shard", s.key) 163 | // shard state isn't active yet; make this acquirer wait. 164 | s.wAcquire = append(s.wAcquire, w) 165 | 166 | // if the shard was registered with lazy init, and this is the 167 | // first acquire, queue the initialization. 168 | if s.state == ShardStateNew { 169 | log.Debugw("acquiring shard with lazy init enabled, will queue shard initialization", "shard", s.key) 170 | // Override the context with the background context. 171 | // We can't use the acquirer's context for initialization 172 | // because there can be multiple concurrent acquirers, and 173 | // if the first one cancels, the entire job would be cancelled. 174 | w := *tsk.waiter 175 | w.ctx = context.Background() 176 | _ = d.queueTask(&task{op: OpShardInitialize, shard: s, waiter: &w}, d.internalCh) 177 | } 178 | 179 | break 180 | } 181 | 182 | // mark as serving. 183 | s.state = ShardStateServing 184 | 185 | // optimistically increment the refcount to acquire the shard. 186 | // The goroutine will send an `OpShardRelease` task 187 | // to the event loop if it fails to acquire the shard. 188 | s.refs++ 189 | go d.acquireAsync(tsk.ctx, w, s, s.mount) 190 | 191 | case OpShardRelease: 192 | if (s.state != ShardStateServing && s.state != ShardStateErrored) || s.refs <= 0 { 193 | log.Warn("ignored illegal request to release shard") 194 | break 195 | } 196 | 197 | // decrement refcount. 198 | s.refs-- 199 | 200 | // reset state back to available, if we were the last 201 | // active acquirer. 202 | if s.refs == 0 { 203 | s.state = ShardStateAvailable 204 | } 205 | 206 | case OpShardFail: 207 | s.state = ShardStateErrored 208 | s.err = tsk.err 209 | 210 | // notify the registration waiter, if there is one. 211 | if s.wRegister != nil { 212 | res := &ShardResult{ 213 | Key: s.key, 214 | Error: fmt.Errorf("failed to register shard: %w", tsk.err), 215 | } 216 | d.dispatchResult(res, s.wRegister) 217 | s.wRegister = nil 218 | } 219 | 220 | // notify the recovery waiter, if there is one. 221 | if s.wRecover != nil { 222 | res := &ShardResult{ 223 | Key: s.key, 224 | Error: fmt.Errorf("failed to recover shard: %w", tsk.err), 225 | } 226 | d.dispatchResult(res, s.wRecover) 227 | s.wRecover = nil 228 | } 229 | 230 | // fail waiting acquirers. 231 | // can't block the event loop, so launch a goroutine per acquirer. 232 | if len(s.wAcquire) > 0 { 233 | err := fmt.Errorf("failed to acquire shard: %w", tsk.err) 234 | res := &ShardResult{Key: s.key, Error: err} 235 | d.dispatchResult(res, s.wAcquire...) 236 | s.wAcquire = s.wAcquire[:0] // clear acquirers. 237 | } 238 | 239 | // Should we interrupt/disturb active acquirers? No. 240 | // 241 | // This part doesn't know which kind of error occurred. 242 | // It could be that the index has disappeared for new acquirers, but 243 | // active acquirers already have it. 244 | // 245 | // If this is a physical error (e.g. shard data was physically 246 | // deleted, or corrupted), we'll leave to the ShardAccessor (and the 247 | // ReadBlockstore) to fail at some point. At that stage, the caller 248 | // will call ShardAccessor#Close and eventually all active 249 | // references will be released, setting the shard in an errored 250 | // state with zero refcount. 251 | 252 | // Notify the application of the failure, if they provided a channel. 253 | if ch := d.failureCh; ch != nil { 254 | res := &ShardResult{Key: s.key, Error: s.err} 255 | d.dispatchFailuresCh <- &dispatch{res: res, w: wFailure} 256 | } 257 | 258 | case OpShardRecover: 259 | if s.state != ShardStateErrored { 260 | err := fmt.Errorf("refused to recover shard in state other than errored; current state: %d", s.state) 261 | res := &ShardResult{Key: s.key, Error: err} 262 | d.dispatchResult(res, tsk.waiter) 263 | break 264 | } 265 | 266 | // set the state to recovering. 267 | s.state = ShardStateRecovering 268 | 269 | // park the waiter; there can never be more than one because 270 | // subsequent calls to recover the same shard will be rejected 271 | // because the state is no longer ShardStateErrored. 272 | s.wRecover = tsk.waiter 273 | 274 | // attempt to delete the transient first; this can happen if the 275 | // transient has been removed by hand. DeleteTransient resets the 276 | // transient to "" always. 277 | if err := s.mount.DeleteTransient(); err != nil { 278 | log.Warnw("recovery: failed to delete transient", "shard", s.key, "error", err) 279 | } 280 | 281 | // attempt to drop the index. 282 | dropped, err := d.indices.DropFullIndex(s.key) 283 | if err != nil { 284 | log.Warnw("recovery: failed to drop index for shard", "shard", s.key, "error", err) 285 | } else if !dropped { 286 | log.Debugw("recovery: no index dropped for shard", "shard", s.key) 287 | } 288 | 289 | // fetch again and reindex. 290 | go d.initializeShard(tsk.ctx, s, s.mount) 291 | 292 | case OpShardDestroy: 293 | if s.state == ShardStateServing || s.refs > 0 { 294 | err := fmt.Errorf("failed to destroy shard; active references: %d", s.refs) 295 | res := &ShardResult{Key: s.key, Error: err} 296 | d.dispatchResult(res, tsk.waiter) 297 | break 298 | } 299 | 300 | d.lk.Lock() 301 | delete(d.shards, s.key) 302 | 303 | // Perform on-disk delete after the switch statement. This is only in-memory delete. 304 | d.lk.Unlock() 305 | res := &ShardResult{Key: s.key, Error: nil} 306 | d.dispatchResult(res, tsk.waiter) 307 | // TODO are we guaranteed that there are no queued items for this shard? 308 | 309 | default: 310 | panic(fmt.Sprintf("unrecognized shard operation: %d", tsk.op)) 311 | 312 | } 313 | 314 | // persist the current shard state. If Op is OpShardDestroy then delete directly from DB. 315 | if tsk.op == OpShardDestroy { 316 | if err := d.store.Delete(d.ctx, datastore.NewKey(s.key.String())); err != nil { 317 | log.Errorw("DestroyShard: failed to delete shard from database", "shard", s.key, "error", err) 318 | } 319 | } else { 320 | if err := s.persist(d.ctx, d.config.Datastore); err != nil { // TODO maybe fail shard? 321 | log.Warnw("failed to persist shard", "shard", s.key, "error", err) 322 | } 323 | } 324 | 325 | // send a notification if the user provided a notification channel. 326 | if d.traceCh != nil { 327 | log.Debugw("will write trace to the trace channel", "shard", s.key) 328 | n := Trace{ 329 | Key: s.key, 330 | Op: tsk.op, 331 | After: ShardInfo{ 332 | ShardState: s.state, 333 | Error: s.err, 334 | refs: s.refs, 335 | }, 336 | } 337 | d.traceCh <- n 338 | log.Debugw("finished writing trace to the trace channel", "shard", s.key) 339 | } 340 | 341 | log.Debugw("finished processing task", "op", tsk.op, "shard", tsk.shard.key, "prev_state", prevState, "curr_state", s.state, "error", tsk.err) 342 | 343 | s.lk.Unlock() 344 | } 345 | } 346 | 347 | func (d *DAGStore) consumeNext() (tsk *task, gc chan *GCResult, error error) { 348 | select { 349 | case tsk = <-d.internalCh: // drain internal first; these are tasks emitted from the event loop. 350 | return tsk, nil, nil 351 | case <-d.ctx.Done(): 352 | return nil, nil, d.ctx.Err() // TODO drain and process before returning? 353 | default: 354 | } 355 | 356 | select { 357 | case tsk = <-d.externalCh: 358 | return tsk, nil, nil 359 | case tsk = <-d.completionCh: 360 | return tsk, nil, nil 361 | case gc := <-d.gcCh: 362 | return nil, gc, nil 363 | case <-d.ctx.Done(): 364 | return nil, nil, d.ctx.Err() // TODO drain and process before returning? 365 | } 366 | } 367 | -------------------------------------------------------------------------------- /dagstore_gc.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | import ( 4 | "io/fs" 5 | "os" 6 | "path/filepath" 7 | 8 | "github.com/filecoin-project/dagstore/shard" 9 | ) 10 | 11 | // GCResult is the result of performing a GC operation. It holds the results 12 | // from deleting unused transients. 13 | type GCResult struct { 14 | // Shards includes an entry for every shard whose transient was reclaimed. 15 | // Nil error values indicate success. 16 | Shards map[shard.Key]error 17 | } 18 | 19 | // ShardFailures returns the number of shards whose transient reclaim failed. 20 | func (e *GCResult) ShardFailures() int { 21 | var failures int 22 | for _, err := range e.Shards { 23 | if err != nil { 24 | failures++ 25 | } 26 | } 27 | return failures 28 | } 29 | 30 | // gc performs DAGStore GC. Refer to DAGStore#GC for more information. 31 | // 32 | // The event loops gives it exclusive execution rights, so while GC is running, 33 | // no other events are being processed. 34 | func (d *DAGStore) gc(resCh chan *GCResult) { 35 | res := &GCResult{ 36 | Shards: make(map[shard.Key]error), 37 | } 38 | 39 | // determine which shards can be reclaimed. 40 | d.lk.RLock() 41 | var reclaim []*Shard 42 | for _, s := range d.shards { 43 | s.lk.RLock() 44 | if nAcq := len(s.wAcquire); (s.state == ShardStateAvailable || s.state == ShardStateErrored) && nAcq == 0 { 45 | reclaim = append(reclaim, s) 46 | } 47 | s.lk.RUnlock() 48 | } 49 | d.lk.RUnlock() 50 | 51 | // attempt to delete transients of reclaimed shards. 52 | for _, s := range reclaim { 53 | // only read lock: we're not modifying state, and the mount has its own lock. 54 | s.lk.RLock() 55 | err := s.mount.DeleteTransient() 56 | if err != nil { 57 | log.Warnw("failed to delete transient", "shard", s.key, "error", err) 58 | } 59 | 60 | // record the error so we can return it. 61 | res.Shards[s.key] = err 62 | 63 | // flush the shard state to the datastore. 64 | if err := s.persist(d.ctx, d.config.Datastore); err != nil { 65 | log.Warnw("failed to persist shard", "shard", s.key, "error", err) 66 | } 67 | s.lk.RUnlock() 68 | } 69 | 70 | select { 71 | case resCh <- res: 72 | case <-d.ctx.Done(): 73 | } 74 | } 75 | 76 | // clearOrphaned removes files that are not referenced by any mount. 77 | // 78 | // This is only safe to be called from the constructor, before we have 79 | // queued tasks. 80 | func (d *DAGStore) clearOrphaned() error { 81 | referenced := make(map[string]struct{}) 82 | 83 | for _, s := range d.shards { 84 | t := s.mount.TransientPath() 85 | referenced[t] = struct{}{} 86 | } 87 | 88 | // Walk the transients dir and delete unreferenced files. 89 | err := filepath.WalkDir(d.config.TransientsDir, func(path string, d fs.DirEntry, err error) error { 90 | if d.IsDir() { 91 | return nil 92 | } 93 | if _, ok := referenced[path]; !ok { 94 | if err := os.Remove(path); err != nil { 95 | log.Warnw("failed to delete orphaned file", "path", path, "error", err) 96 | } else { 97 | log.Infow("deleted orphaned file", "path", path) 98 | } 99 | } 100 | return nil 101 | }) 102 | 103 | return err 104 | } 105 | -------------------------------------------------------------------------------- /dispatcher.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | // dispatcher takes care of dispatching results back to the application. 4 | // 5 | // These can be results of API operations, or shard failures. 6 | func (d *DAGStore) dispatcher(ch chan *dispatch) { 7 | defer d.wg.Done() 8 | 9 | var di *dispatch 10 | for { 11 | select { 12 | case di = <-ch: 13 | case <-d.ctx.Done(): 14 | return 15 | } 16 | di.w.deliver(di.res) 17 | } 18 | } 19 | 20 | func (d *DAGStore) dispatchResult(res *ShardResult, waiters ...*waiter) { 21 | for _, w := range waiters { 22 | if w.outCh == nil { 23 | // no return channel; skip. 24 | continue 25 | } 26 | d.dispatchResultsCh <- &dispatch{w: w, res: res} 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /docs/design.md: -------------------------------------------------------------------------------- 1 | ## CARv2 + DAG store implementation notes 2 | 3 | ## Overview 4 | 5 | The purpose of the CARv2 + DAG store endeavour is to eliminate overhead from the 6 | deal-making processes with the mission of unlocking scalability, performance, 7 | and resource frugality on both miner and client side within the Filecoin 8 | network. 9 | 10 | Despite serving Lotus/Filecoin immediate needs, we envision the DAG store to be 11 | a common interplanetary building block across IPFS, Filecoin, and IPLD-based 12 | projects. 13 | 14 | The scalability of the former two is bottlenecked by the usage of Badger as a 15 | monolithic blockstore. The DAG store represents an alternative that leverages 16 | sharding and the concept of detachable "data cartridges" to feed and manage data 17 | as transactional units. 18 | 19 | For the project that immediately concerns us, the definition of done is the 20 | removal of the Badger blockstore from all client and miner codepaths involved in 21 | storage and retrieval deals, and the transition towards interfacing directly 22 | with low-level data repository files known as CARs (Content ARchives), on both 23 | the read and the write sides. 24 | 25 | Note that Badger is generally a fitting database for write-intense and 26 | iterator-heavy workloads. It just doesn't withstand the pattern of usage we 27 | subject it to in large IPLD-based software components, especially past the 100s 28 | GBs data volume mark. 29 | 30 | For more information on motivation and architecture, please refer to 31 | [CAR-native DAG store](https://docs.google.com/document/d/118fJdf8HGK8tHYOCQfMVURW9gTF738D0weX-hbG-BvY/edit#). This document is 32 | recommended as a pre-read. 33 | 34 | ## Overview 35 | 36 | The DAG store comprises three layers: 37 | 38 | 1. Storage layer (manages shards) 39 | 2. Index repository (manages indices) 40 | 3. DAG access layer (manages queries) 41 | 42 | ## Storage layer 43 | 44 | The DAG store holds shards of data. In Filecoin, shard = deal. Each shard is a 45 | repository of data capable of acting as a standalone blockstore. 46 | 47 | ### Shards 48 | 49 | Shards are identified by an opaque byte string: the shard key. In the case of 50 | Filecoin, the shard key is the `PieceCID` of the storage deal. 51 | 52 | A shard contains: 53 | 54 | 1. the shard key (identifier). 55 | 2. the means to obtain shard data (a `Mount` that provides the CAR either 56 | locally, remotely, or some other location). 57 | 3. the shard index (both acting as a manifest of the contents of the shard, and 58 | a means to efficiently perform random reads). 59 | 60 | ### CAR formats 61 | 62 | A shard can be filled with CARv1 and CARv2 data. CARv2 can be indexed or 63 | indexless. 64 | 65 | The choice of version and characteristics affects how the shard index is 66 | populated: 67 | 68 | 1. CARv1: the index is calculated upon shard activation. 69 | 2. Indexless CARv2: the index is calculated upon shard activation. 70 | 3. Indexed CARv2: the inline index is adopted as-is as the shard index. 71 | 72 | ### Shard states 73 | 74 | 1. **Available:** the system knows about this shard and is capable of serving 75 | data from it instantaneously, because (a) an index exists and is loaded, and 76 | (b) data is locally accessible (e.g. it doesn't need to be fetched from a 77 | remote mount). 78 | 2. **Unavailable:** the system knows about this shard, but is not capable of 79 | serving data from it because the shard is being initialized, or the mount is 80 | not available locally, but still accessible with work (e.g. fetching from a 81 | remote location). 82 | 3. **Destroyed**: the shard is no longer available; this is permanent condition. 83 | 84 | ### Operations 85 | 86 | #### Shard registration 87 | 88 | To register a new shard, call: 89 | 90 | ```go 91 | dagst.RegisterShard(key []byte, mount Mount, opts ...RegisterOpts) error 92 | ``` 93 | 94 | 1. This method takes a shard key and a `Mount`. 95 | 2. It initializes the shard in `Unavailable` state. 96 | 3. Calls `mount.Info()` to determine if the mount is of local or remote type. 97 | 4. Calls `mount.Stat()` to determine if the mount target exists. If not, it 98 | errors the registration. 99 | 5. If remote, it fetches the remote resource into the scrap area. 100 | 6. It determines the kind of CAR it is, and populates the index accordingly. 101 | 7. Sets the state to `Available`. 102 | 8. Returns. 103 | 104 | This method is _synchronous_. It returns only when the shard is fully indexed 105 | and available for serving data. This embodies the consistency property of the 106 | ACID model, whereby an INSERT is immediately queriable upon return. 107 | 108 | _RegisterOpts is an extension point._ In the future it can be used to pass in an 109 | unseal/decryption function to be used on access (e.g. such as when fast, random 110 | unseal is available). 111 | 112 | #### Shard destruction 113 | 114 | To destroy a shard, call: 115 | 116 | ```go 117 | dagst.DestroyShard(key []byte) (destroyed bool, err error) 118 | ``` 119 | 120 | This erases transient copies from the scrap area, and it removes the shard/mount 121 | from the shard persistence store. It errors if the shard is locked/acquired. 122 | 123 | #### Other operations 124 | 125 | * `[Pin/Unpin]Shard()`: retains the shard data in the local scrap area. 126 | * `ReleaseShard()`: dispose of / release local scrap copies or other resources 127 | on demand (e.g. unsealed copy). 128 | * `[Lock/Unlock]Shard()`: for exclusive access. 129 | * `[Incr/Decr]Shard()`: refcounting on shards. 130 | 131 | ### Mounts 132 | 133 | Shards can be located anywhere, and can come and go dynamically e.g. Filecoin 134 | deals expire, removable media is attached/detached, or the IPFS user purges 135 | content. 136 | 137 | It is possible to mount shards with CARs accessible through the local 138 | filesystem, detachable mounts, NFS mounts, distributed filesystems like 139 | Ceph/GlusterFS, HTTP, FTP, etc. 140 | 141 | This versatility is provided by an abstraction called `mount.Mount`, a pluggable 142 | component which encapsulates operations/logic to: 143 | 144 | 1. `Fetch() (io.ReadSeekCloser, error)` Load a CAR from its origin. 145 | 2. `Info() mount.Info` Provides info about the mount, e.g. whether it's local or 146 | remote. This is used to determine whether the fetched CAR needs to be copied 147 | to a scrap area. It also indicates whether the stream supports seeking. 148 | Calling `Seek()` on an unseekable stream will panic. `Seek()` is used to 149 | fast-forward to an index if only that structure needs to be consumed. 150 | 3. `Stat() (mount.Stat, error)` Equivalent to a filesystem stat, provides 151 | information about the target of the mount: whether it exists, size, etc. 152 | 153 | When instantiating `Mount` implementations, one can provide credentials, access 154 | tokens, or other parameters through the implementation constructors. This is 155 | necessary if access to the CAR is permissioned/authenticated. 156 | 157 | **Local filesystem Mount** 158 | 159 | A local filesystem `Mount` implementation loads the CAR directly from the 160 | filesystem file. It is of `local` type and therefore requires no usage of scrap 161 | area. 162 | 163 | *This `Mount` is provided out-of-box by the DAG store.* 164 | 165 | **Lotus Mount implementation** 166 | 167 | A Lotus `Mount` implementation would be instantiated with a sector ID and a 168 | bytes range within the sealed sector file (i.e. the deal segment). 169 | 170 | Loading the CAR consists of calling the worker HTTP API to fetch the unsealed 171 | piece. Because the mount is of `remote` type, the DAG store will need to store 172 | it in a local scrap area. Currently, this may lead to actual unsealing on the 173 | Lotus worker cluster through the current PoRep (slow) if the piece is sealed. 174 | 175 | With a future PoRep (cheap, snappy) PoRep, sealing can be performed _just_ for 176 | the blocks that are effectively accessed, potentially during IPLD block access 177 | time. A transformation function may be provided in the future as a `RegisterOpt` 178 | that conducts the unsealing. 179 | 180 | A prerequisite to enable unsealing-on-demand possible is PoRep and index 181 | symmetry, i.e. the byte positions of blocks in the sealed CAR must be 182 | identical to those in the unsealed CAR. 183 | 184 | *This `Mount` is provided by Lotus, as it's implementation specific.* 185 | 186 | #### URL representation and registry 187 | 188 | Mounts are stateless objects, serialized as URL for persistence and 189 | configuration. This enables us to: 190 | 191 | 1. Record mounts in shard persistence (see below). 192 | 2. Add shards by configuration (e.g. imagine a program whose shards are 193 | configured in a configuration file) 194 | 195 | The URL format mapping is: 196 | 197 | ``` 198 | scheme://host[:port]?key1=value1&key2=value2[&...] 199 | ``` 200 | 201 | - `scheme` is a unique identifier for the mount type, e.g. `lotus://`, 202 | `file://`, `http://`. 203 | - the rest is scheme-dependent. 204 | - `host[:port]` is usually the main component of the mount, e.g. a file path, 205 | a sector ID + deal ID, etc. 206 | - query parameters map to mount options (e.g. credentials, more) 207 | 208 | Scheme -> implementation bindings are kept in a registry. 209 | 210 | ### Shard representation and persistence 211 | 212 | The shard catalogue needs to survive restarts. Thus, it needs to be persistent. 213 | Options to explore here include LMDB, BoltDB, or others. Here's what the 214 | persisted shard entry could look like: 215 | 216 | ```go 217 | type PersistedShard struct { 218 | Key []byte 219 | // Mount is a URL representation of the Mount, e.g. 220 | // file://path/to/file?opt=1&opt=2 221 | // lotus://sectorID?offset=1&length=2 222 | Mount string 223 | // LocalPath is the path to the local replica of the CAR in the scrap area, 224 | // if the mount if of remote type. 225 | LocalPath string 226 | } 227 | ``` 228 | 229 | Upon starting, the DAG store will load the catalogue from disk and will 230 | reinstantiate the shard catalogue, the mounts, and the shard states. 231 | 232 | ### Scrap area 233 | 234 | When dealing with remote mounts (e.g. Filecoin storage cluster), the DAG store 235 | will need to copy the remote CAR into local storage to be able to serve DAG 236 | access queries. These copies are called _transient copies_. 237 | 238 | Readers access shards from the storage layer by calling 239 | `Acquire/ReleaseShard(key []byte)` methods, which drive the copies into scrap 240 | storage and the deletion of resources. 241 | 242 | These methods will need to do refcounting. When no readers are accessing a 243 | shard, the DAG store is free to release local resources. In a first version, 244 | this may happen instantly. In future versions, we may introduce some active 245 | management of the scrap area through usage monitoring + GC. Storage space 246 | assigned to the scrap area may by configuration in the future. 247 | 248 | ## Index repository 249 | 250 | The index repository is the subcomponent that owns and manages the indices in 251 | the DAG store. 252 | 253 | There exists three kinds of indices: 254 | 255 | 1. **Full shard indices.** Consisting of `{ CID: offset }` mappings. In indexed 256 | CARv2, these are extracted from the inline indices. In unindexed CARv2 and 257 | CARv1, these are computed using the [Carbs library](https://github.com/willscott/carbs), or the CARv2 upgrade path. 258 | 259 | Full shard indices are protected, and not writable externally. Every 260 | available/unavailable shard MUST have a full shard index. Upon shard 261 | destruction, its associated full shard index can be disposed. 262 | 263 | 2. **Semantic indices.** Manifests of externally-relevant CIDs contained within 264 | a shard, i.e. `[]CID` with no offset indication. In other words, subsets of 265 | the full-shard index with no offset indication. 266 | 267 | These are calculated externally (e.g. semantic indexing service) and supplied 268 | to the DAG store for storage and safekeeping. 269 | 270 | A shard can have infinite number of semantic indices associated with it. Each 271 | semantic index is identified and stamped with its generation data (rule and 272 | version). 273 | 274 | We acknowledge that semantic indices perhaps don't belong in the DAG store 275 | long-term. Despite that, we decide to incubate them here to potentially spin 276 | them off in the future. 277 | 278 | 3. **Top-level cross-shard index.** Aggregates of full shard indices that enable 279 | shard routing of reads for concrete CIDs. 280 | 281 | ### Interactions 282 | 283 | This component receives the queries coming in from the miner-side indexing 284 | sidecar, which in turn come from network indexers. 285 | 286 | It also serves the DAG access layer. When a shard is registered/acquired in the 287 | storage layer, and a Blockstore is demanded for it, the full index to provide 288 | random-access capability is obtained from the index repo. 289 | 290 | In the future, this subcomponent will also serve the top-level cross-shard 291 | index. 292 | 293 | ### Persistence and reconciliation 294 | 295 | Indices will be persisted in a filesystem directory, referenced by their shard 296 | key. This makes for easy identification. 297 | 298 | Although not expected, it is possible that over time the index repo and the 299 | shard management layer state will diverge (entropy): 300 | 301 | 1. Active shards may have their indices deleted (e.g. if an operator deleted a 302 | file or the directory). 303 | 2. Destroyed shards may have their indices lying around (e.g. if there was some 304 | system error in housekeeping). 305 | 306 | A method `ReconcileIndices()` can be used to reconcile the index repo with the 307 | shard management layer, by: 308 | 309 | 1. Fetching gone indices from their mounts. 310 | 2. Deleting lingering indices. 311 | 312 | ### Interface 313 | 314 | ```go 315 | type IndexRepo interface { 316 | FullIndexRepo 317 | ManifestRepo 318 | } 319 | 320 | type FullIndexRepo interface { 321 | // public 322 | GetFullIndex(key shard.Key) (FullIndex, error) 323 | 324 | // private, called only by shard management when registering 325 | // and destroying shards 326 | InsertFullIndex(key shard.Key, index FullIndex) error 327 | DeleteFullIndex(key shard.Key) (bool, error) 328 | } 329 | 330 | type ManifestRepo interface { 331 | // public 332 | GetManifest(key ManifestKey) (Manifest, error) 333 | InsertManifest(key ManifestKey, manifest Manifest) error 334 | DeleteManifest(key ManifestKey) (bool, error) 335 | ExistsManifest(key ManifestKey) (bool, error) 336 | } 337 | 338 | type ManifestKey struct { 339 | Shard key.Shard 340 | Rule string 341 | Version string 342 | } 343 | 344 | type FullIndex interface { 345 | Offset(c cid.Cid) (offset int64, err error) // inexistent: offset=-1, err=nil 346 | Contains(c cid.Cid) (bool, error) 347 | Len() (l int64, err error) 348 | ForEach(func(c cid.Cid, offset int64) (ok bool, err error)) error 349 | } 350 | ``` 351 | 352 | ## DAG access layer 353 | 354 | This layer is responsible for serving DAGs or sub-DAGs from one or many shards. 355 | Initially, queries will require the shard key. That is, queries will be bounded 356 | to a single identifiable shard. 357 | 358 | In the future, when the cross-shard top-level index is implemented, the DAG 359 | store will be capable of resolving the shard for any given root CID. 360 | 361 | DAG access layer allows various access abstractions: 362 | 1. Obtaining a Blockstore bound to a single shard. 363 | 2. Obtaining a Blockstore bound to a specific set of shards. 364 | 3. Obtaining a global Blockstore. 365 | 4. `ipld.Node` -- TBD. 366 | 367 | At this point, we are only concerned with (1). The remaining access patterns 368 | will be elaborated on in the future. 369 | 370 | ```go 371 | type DAGAccessor interface { 372 | Shard(key shard.Key) ShardAccessor 373 | } 374 | 375 | type ShardAccessor interface { 376 | // Close() the accessor, signalling to the DAG store that we're done here and 377 | // possible transient copies can be reclaimed once the shard is released. 378 | io.Closer 379 | 380 | Blockstore() (ReadBlockstore, error) 381 | } 382 | ``` 383 | 384 | ## Requirements for CARv2 385 | 386 | - Index needs to be detachable. 387 | - Index offsets need to be relative to the CARv1, and not absolute in the 388 | physical CARv2 file. 389 | - Index needs to be iterable. 390 | - Given any CAR file, we should be able to determine its version and 391 | characteristics (concretely, indexed or not indexed). 392 | - Given a CARv1 file, it should be possible to generate a CARv2 index for it in 393 | detached form. 394 | - Given a CARv2 file, we should be able to decompose it to the corresponding 395 | CARv1 payload and the Carbs index. The CAR payload should be exposed with 396 | `io.ReaderAt` and `io.Reader` abstractions. 397 | - Given an indexed CARv2 file, it should be possible to instantiate a 398 | `ReadBlockstore` on it in a self-sufficient manner. 399 | - Given a CARv1 or unindexed CARv2, it should be possible to instantiate a 400 | `ReadBlockstore` with an index provided by us. 401 | - It should be possible to write an indexed CARv2 in streaming fashion. The 402 | CARv2 library should do the right thing depending on the declared write 403 | characteristics and the output characteristics. For example: 404 | 1. the writer may declare that blocks are provided in depth-first order and 405 | without uniqueness guarantees, but may state they want the output to be in 406 | depth-first order AND deduplicated. In this case, the CARv2 library must 407 | use the index to prevent duplicate writes (this is the priority case right 408 | now). 409 | 2. the writer may declare blocks are provided in no order and without 410 | uniqueness guarantees, but may wish to produce an output in depth-first 411 | order and deduplicated. In this case, when finalizing the CAR, the library 412 | must evaluate if a rewrite is necessary, and, if so, it should conduct it 413 | (this case is not a priority now, but needed down the line for feature 414 | completeness). 415 | - When a client finishes writing to a CARv2 blockstore, it should be able to 416 | finish/flush/commit the write and get a CARv2 file. 417 | - A CARv2 SelectiveCAR API that looks/feels/behaves exactly like a CARv1 Selective 418 | CAR API but outputs/writes a v2 instead of a v1 (Note: the output CARv2 file 419 | should NOT have duplicate blocks -> this behaviour is similar to what we have 420 | for the v1 APIs). 421 | - Index needs to be marshalable/unmarshallable so the DAG Store can persist/read 422 | it. 423 | 424 | ## Discussion: migration from current architecture 425 | 426 | TBD. 427 | 428 | ## Open points 429 | 430 | - ... 431 | - ... 432 | - ... 433 | -------------------------------------------------------------------------------- /gen/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "fmt" 5 | "os" 6 | 7 | "github.com/filecoin-project/dagstore" 8 | gen "github.com/whyrusleeping/cbor-gen" 9 | ) 10 | 11 | func main() { 12 | err := gen.WriteMapEncodersToFile("./shard_gen.go", "dagstore", 13 | dagstore.PersistedShard{}, 14 | ) 15 | if err != nil { 16 | fmt.Println(err) 17 | os.Exit(1) 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /go.mod: -------------------------------------------------------------------------------- 1 | module github.com/filecoin-project/dagstore 2 | 3 | go 1.16 4 | 5 | require ( 6 | github.com/ipfs/go-block-format v0.0.3 7 | github.com/ipfs/go-blockservice v0.5.0 8 | github.com/ipfs/go-cid v0.3.2 9 | github.com/ipfs/go-cidutil v0.1.0 10 | github.com/ipfs/go-datastore v0.6.0 11 | github.com/ipfs/go-ds-leveldb v0.5.0 12 | github.com/ipfs/go-ipfs-blockstore v1.2.0 13 | github.com/ipfs/go-ipfs-blocksutil v0.0.1 14 | github.com/ipfs/go-ipfs-chunker v0.0.1 15 | github.com/ipfs/go-ipfs-exchange-offline v0.3.0 16 | github.com/ipfs/go-ipfs-files v0.0.3 17 | github.com/ipfs/go-ipld-format v0.3.0 18 | github.com/ipfs/go-log/v2 v2.5.1 19 | github.com/ipfs/go-merkledag v0.8.1 20 | github.com/ipfs/go-unixfs v0.3.1 21 | github.com/ipld/go-car/v2 v2.4.1 22 | github.com/jellydator/ttlcache/v2 v2.11.1 23 | github.com/mr-tron/base58 v1.2.0 24 | github.com/multiformats/go-multicodec v0.5.0 25 | github.com/multiformats/go-multihash v0.2.1 26 | github.com/stretchr/testify v1.8.0 27 | github.com/syndtr/goleveldb v1.0.0 28 | github.com/whyrusleeping/cbor-gen v0.0.0-20200123233031-1cdf64d27158 29 | golang.org/x/exp v0.0.0-20210714144626-1041f73d31d8 30 | golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4 31 | golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 32 | ) 33 | -------------------------------------------------------------------------------- /handlers.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/filecoin-project/dagstore/shard" 7 | ) 8 | 9 | // RecoverImmediately takes a failureCh where DAGStore failures are sent, and 10 | // attempts to recover the shard immediately up until maxAttempts for each 11 | // unique shard. 12 | // 13 | // Attempt tracking does not survive restarts. When the passed context fires, 14 | // the failure handler will yield and the given `onDone` function is called before returning. It is recommended to call this 15 | // method from a dedicated goroutine, as it runs an infinite event 16 | // loop. 17 | func RecoverImmediately(ctx context.Context, dagst *DAGStore, failureCh chan ShardResult, maxAttempts uint64, onDone func()) { 18 | if onDone != nil { 19 | defer onDone() 20 | } 21 | var ( 22 | recResCh = make(chan ShardResult, 128) 23 | attempts = make(map[shard.Key]uint64) 24 | ) 25 | 26 | for { 27 | select { 28 | case res := <-failureCh: 29 | key := res.Key 30 | att := attempts[key] 31 | if att >= maxAttempts { 32 | log.Infow("failure handler: max attempts exceeded; skipping recovery", "key", key, "from_error", res.Error, "attempt", att) 33 | continue 34 | } 35 | 36 | log.Infow("failure handler: recovering shard", "key", key, "from_error", res.Error, "attempt", att) 37 | 38 | // queue the recovery for this key. 39 | if err := dagst.RecoverShard(ctx, key, recResCh, RecoverOpts{}); err != nil { 40 | log.Warnw("failure handler: failed to queue shard recovery", "key", key, "error", err) 41 | continue 42 | } 43 | attempts[key]++ 44 | 45 | case res := <-recResCh: 46 | // this channel is just informational; a failure to recover will 47 | // trigger another failure on the failureCh, which will be handled 48 | // above for retry. 49 | key := res.Key 50 | if res.Error == nil { 51 | log.Infow("failure handler: successfully recovered shard", "key", key) 52 | delete(attempts, key) 53 | } else { 54 | log.Warnw("failure handler: failed to recover shard", "key", key, "attempt", attempts[key]) 55 | } 56 | continue 57 | 58 | case <-ctx.Done(): 59 | log.Info("failure handler: stopping") 60 | attempts = nil 61 | return 62 | } 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /index/inverted_index_impl.go: -------------------------------------------------------------------------------- 1 | package index 2 | 3 | import ( 4 | "context" 5 | "encoding/json" 6 | "fmt" 7 | "sync" 8 | 9 | "github.com/ipfs/go-datastore/namespace" 10 | 11 | ds "github.com/ipfs/go-datastore" 12 | 13 | "github.com/multiformats/go-multihash" 14 | 15 | "github.com/filecoin-project/dagstore/shard" 16 | ) 17 | 18 | var _ Inverted = (*invertedIndexImpl)(nil) 19 | 20 | type invertedIndexImpl struct { 21 | mu sync.Mutex 22 | ds ds.Batching 23 | } 24 | 25 | // NewInverted returns a new inverted index that uses `go-indexer-core` 26 | // as it's storage backend. We use `go-indexer-core` as the backend here 27 | // as it's been optimized to store (multihash -> Value) kind of data and 28 | // supports bulk updates via context ID and metadata-deduplication which are useful properties for our use case here. 29 | func NewInverted(dts ds.Batching) *invertedIndexImpl { 30 | dts = namespace.Wrap(dts, ds.NewKey("/inverted/index")) 31 | return &invertedIndexImpl{ 32 | ds: dts, 33 | } 34 | } 35 | 36 | func (d *invertedIndexImpl) AddMultihashesForShard(ctx context.Context, mhIter MultihashIterator, s shard.Key) error { 37 | d.mu.Lock() 38 | defer d.mu.Unlock() 39 | 40 | batch, err := d.ds.Batch(ctx) 41 | if err != nil { 42 | return fmt.Errorf("failed to create ds batch: %w", err) 43 | } 44 | 45 | if err := mhIter.ForEach(func(mh multihash.Multihash) error { 46 | key := ds.NewKey(string(mh)) 47 | // do we already have an entry for this multihash ? 48 | val, err := d.ds.Get(ctx, key) 49 | if err != nil && err != ds.ErrNotFound { 50 | return fmt.Errorf("failed to get value for multihash %s, err: %w", mh, err) 51 | } 52 | 53 | // if we don't have an existing entry for this mh, create one 54 | if err == ds.ErrNotFound { 55 | s := []shard.Key{s} 56 | bz, err := json.Marshal(s) 57 | if err != nil { 58 | return fmt.Errorf("failed to marshal shard list to bytes: %w", err) 59 | } 60 | if err := batch.Put(ctx, key, bz); err != nil { 61 | return fmt.Errorf("failed to put mh=%s, err=%w", mh, err) 62 | } 63 | return nil 64 | } 65 | 66 | // else , append the shard key to the existing list 67 | var es []shard.Key 68 | if err := json.Unmarshal(val, &es); err != nil { 69 | return fmt.Errorf("failed to unmarshal shard keys: %w", err) 70 | } 71 | 72 | // if we already have the shard key indexed for the multihash, nothing to do here. 73 | if has(es, s) { 74 | return nil 75 | } 76 | 77 | es = append(es, s) 78 | bz, err := json.Marshal(es) 79 | if err != nil { 80 | return fmt.Errorf("failed to marshal shard keys: %w", err) 81 | } 82 | if err := batch.Put(ctx, key, bz); err != nil { 83 | return fmt.Errorf("failed to put mh=%s, err%w", mh, err) 84 | } 85 | 86 | return nil 87 | }); err != nil { 88 | return fmt.Errorf("failed to add index entry: %w", err) 89 | } 90 | 91 | if err := batch.Commit(ctx); err != nil { 92 | return fmt.Errorf("failed to commit batch: %w", err) 93 | } 94 | 95 | if err := d.ds.Sync(ctx, ds.Key{}); err != nil { 96 | return fmt.Errorf("failed to sync puts: %w", err) 97 | } 98 | 99 | return nil 100 | } 101 | 102 | func (d *invertedIndexImpl) GetShardsForMultihash(ctx context.Context, mh multihash.Multihash) ([]shard.Key, error) { 103 | key := ds.NewKey(string(mh)) 104 | sbz, err := d.ds.Get(ctx, key) 105 | if err != nil { 106 | return nil, fmt.Errorf("failed to lookup index for mh %s, err: %w", mh, err) 107 | } 108 | 109 | var shardKeys []shard.Key 110 | if err := json.Unmarshal(sbz, &shardKeys); err != nil { 111 | return nil, fmt.Errorf("failed to unmarshal shard keys for mh=%s, err=%w", mh, err) 112 | } 113 | 114 | return shardKeys, nil 115 | } 116 | 117 | func has(es []shard.Key, k shard.Key) bool { 118 | for _, s := range es { 119 | if s == k { 120 | return true 121 | } 122 | } 123 | return false 124 | } 125 | -------------------------------------------------------------------------------- /index/inverted_index_impl_test.go: -------------------------------------------------------------------------------- 1 | package index 2 | 3 | import ( 4 | "context" 5 | "testing" 6 | 7 | blocksutil "github.com/ipfs/go-ipfs-blocksutil" 8 | 9 | levelds "github.com/ipfs/go-ds-leveldb" 10 | ldbopts "github.com/syndtr/goleveldb/leveldb/opt" 11 | 12 | "github.com/ipfs/go-datastore/sync" 13 | 14 | ds "github.com/ipfs/go-datastore" 15 | 16 | "golang.org/x/xerrors" 17 | 18 | "github.com/ipfs/go-cid" 19 | "github.com/multiformats/go-multihash" 20 | "github.com/stretchr/testify/require" 21 | 22 | "github.com/filecoin-project/dagstore/shard" 23 | ) 24 | 25 | var blockGenerator = blocksutil.NewBlockGenerator() 26 | 27 | func TestDatastoreIndexEmpty(t *testing.T) { 28 | ctx := context.Background() 29 | req := require.New(t) 30 | 31 | cid1, err := cid.Parse("Qmard76Snyj9VCJBzLSLYzXnJJ2BnyCN2KAfAkpLXyt1q7") 32 | req.NoError(err) 33 | 34 | idx := NewInverted(sync.MutexWrap(ds.NewMapDatastore())) 35 | 36 | _, err = idx.GetShardsForMultihash(ctx, cid1.Hash()) 37 | req.True(xerrors.Is(err, ds.ErrNotFound)) 38 | } 39 | 40 | func TestLevelDBBatch(t *testing.T) { 41 | ctx := context.Background() 42 | // Create a new LevelDB datastore 43 | dstore, err := levelds.NewDatastore(t.TempDir(), &levelds.Options{ 44 | Compression: ldbopts.NoCompression, 45 | NoSync: false, 46 | Strict: ldbopts.StrictAll, 47 | ReadOnly: false, 48 | }) 49 | require.NoError(t, err) 50 | idx := NewInverted(dstore) 51 | 52 | // add 100,000 multihashes 53 | mhs := GenerateMhs(100000) 54 | require.Len(t, mhs, 100000) 55 | itIdxA := &mhIt{mhs} 56 | sk1 := shard.KeyFromString("shard-key-1") 57 | err = idx.AddMultihashesForShard(ctx, itIdxA, sk1) 58 | require.NoError(t, err) 59 | 60 | for _, mh := range mhs { 61 | sk, err := idx.GetShardsForMultihash(ctx, mh) 62 | require.NoError(t, err) 63 | require.Len(t, sk, 1) 64 | require.Contains(t, sk, sk1) 65 | } 66 | } 67 | 68 | func TestDatastoreIndex(t *testing.T) { 69 | ctx := context.Background() 70 | req := require.New(t) 71 | 72 | cid1, err := cid.Parse("Qmard76Snyj9VCJBzLSLYzXnJJ2BnyCN2KAfAkpLXyt1q7") 73 | req.NoError(err) 74 | cid2, err := cid.Parse("Qmard76Snyj9VCJBzLSLYzXnJJ2BnyCN2KAfAkpLXyt1q8") 75 | req.NoError(err) 76 | cid3, err := cid.Parse("Qmard76Snyj9VCJBzLSLYzXnJJ2BnyCN2KAfAkpLXyt1q9") 77 | req.NoError(err) 78 | 79 | h1 := cid1.Hash() 80 | h2 := cid2.Hash() 81 | h3 := cid3.Hash() 82 | 83 | idx := NewInverted(sync.MutexWrap(ds.NewMapDatastore())) 84 | 85 | // Add hash to shard key mappings for h1, h2: also dedupes the multihash iterator 86 | // h1 -> [shard-key-1] 87 | // h2 -> [shard-key-1] 88 | itIdxA := &mhIt{[]multihash.Multihash{h1, h2, h1, h1}} 89 | sk1 := shard.KeyFromString("shard-key-1") 90 | err = idx.AddMultihashesForShard(ctx, itIdxA, sk1) 91 | req.NoError(err) 92 | 93 | // Add hash to shard key mappings for h1, h3: 94 | // h1 -> [shard-key-1, shard-key-2] 95 | // h3 -> [shard-key-2] 96 | itIdxB := &mhIt{[]multihash.Multihash{h1, h3}} 97 | sk2 := shard.KeyFromString("shard-key-2") 98 | err = idx.AddMultihashesForShard(ctx, itIdxB, sk2) 99 | req.NoError(err) 100 | 101 | // add shard-key-1 again for h1 -> will get de-duped 102 | itIdxC := &mhIt{[]multihash.Multihash{h1}} 103 | err = idx.AddMultihashesForShard(ctx, itIdxC, sk1) 104 | req.NoError(err) 105 | 106 | // Verify h1 mapping: 107 | // h1 -> [shard-key-1, shard-key-2] 108 | shards, err := idx.GetShardsForMultihash(ctx, cid1.Hash()) 109 | req.NoError(err) 110 | req.Len(shards, 2) 111 | req.Contains(shards, sk1) 112 | req.Contains(shards, sk2) 113 | 114 | // Verify h2 mapping: 115 | // h2 -> [shard-key-1] 116 | shards, err = idx.GetShardsForMultihash(ctx, cid2.Hash()) 117 | req.NoError(err) 118 | req.Len(shards, 1) 119 | req.Equal(shards[0], sk1) 120 | } 121 | 122 | type mhIt struct { 123 | mhs []multihash.Multihash 124 | } 125 | 126 | var _ MultihashIterator = (*mhIt)(nil) 127 | 128 | func (mi *mhIt) ForEach(f func(mh multihash.Multihash) error) error { 129 | for _, mh := range mi.mhs { 130 | if err := f(mh); err != nil { 131 | return err 132 | } 133 | } 134 | return nil 135 | } 136 | 137 | // GenerateMhs produces n mutlihashes. 138 | func GenerateMhs(n int) []multihash.Multihash { 139 | mhs := make([]multihash.Multihash, 0, n) 140 | for i := 0; i < n; i++ { 141 | c := blockGenerator.Next().Cid() 142 | mhs = append(mhs, c.Hash()) 143 | } 144 | return mhs 145 | } 146 | -------------------------------------------------------------------------------- /index/inverted_index_interface.go: -------------------------------------------------------------------------------- 1 | package index 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/filecoin-project/dagstore/shard" 7 | "github.com/multiformats/go-multihash" 8 | ) 9 | 10 | type MultihashIterator interface { 11 | ForEach(func(mh multihash.Multihash) error) error 12 | } 13 | 14 | // Inverted is the top-level inverted index that maps a multihash to all the shards it is present in. 15 | type Inverted interface { 16 | // AddMultihashesForShard adds a (multihash -> shard key) mapping for all multihashes returned by the given MultihashIterator. 17 | AddMultihashesForShard(ctx context.Context, mhIter MultihashIterator, s shard.Key) error 18 | // GetShardsForMultihash returns keys for all the shards that has the given multihash. 19 | GetShardsForMultihash(ctx context.Context, h multihash.Multihash) ([]shard.Key, error) 20 | } 21 | -------------------------------------------------------------------------------- /index/manifest.go: -------------------------------------------------------------------------------- 1 | package index 2 | 3 | import ( 4 | "github.com/filecoin-project/dagstore/shard" 5 | "github.com/ipfs/go-cid" 6 | ) 7 | 8 | // ManifestKey identifies a manifest. It is a triple that can act as a composite 9 | // key, comprising the shard key and generation metadata. 10 | type ManifestKey struct { 11 | Shard shard.Key 12 | GenRule string 13 | GenVersion uint 14 | } 15 | 16 | // Manifest are sets of CIDs with no offset indication. 17 | type Manifest interface { 18 | // Contains checks whether a given CID is contained in the manifest. 19 | Contains(c cid.Cid) (bool, error) 20 | 21 | // Len returns the count of entries this manifest has. 22 | Len() (l int64, err error) 23 | 24 | // ForEach traverses the manifest using an visitor pattern. The supplied 25 | // callback will be called for each manifest entry, in no particular order. 26 | // 27 | // Returning true from the callback will continue the traversal. 28 | // Returning false will terminate the traversal. 29 | // 30 | // A non-nil error will abort the traversal, and the error will be 31 | // propagated to the caller. 32 | ForEach(func(c cid.Cid) (ok bool, err error)) error 33 | } 34 | -------------------------------------------------------------------------------- /index/repo.go: -------------------------------------------------------------------------------- 1 | package index 2 | 3 | import ( 4 | "errors" 5 | 6 | "github.com/filecoin-project/dagstore/shard" 7 | "github.com/ipld/go-car/v2/index" 8 | ) 9 | 10 | var ErrNotFound = errors.New("index not found") 11 | 12 | // Repo is the central index repository object that manages full indices and 13 | // manifests. 14 | type Repo interface { 15 | FullIndexRepo 16 | ManifestRepo 17 | } 18 | 19 | type Stat struct { 20 | Exists bool 21 | Size uint64 22 | } 23 | 24 | type FullIndexRepo interface { 25 | // GetFullIndex returns the full index for the specified shard. 26 | GetFullIndex(key shard.Key) (index.Index, error) 27 | 28 | // AddFullIndex persists a full index for a shard. 29 | AddFullIndex(key shard.Key, index index.Index) error 30 | 31 | // DropFullIndex drops the full index for the specified shard. If the error 32 | // is nil, it returns whether an index was effectively dropped. 33 | DropFullIndex(key shard.Key) (dropped bool, err error) 34 | 35 | // StatFullIndex stats a full index. 36 | StatFullIndex(key shard.Key) (Stat, error) 37 | 38 | // Len returns the number of indices in the repo. 39 | Len() (int, error) 40 | 41 | // ForEach calls the callback with the key for each index. 42 | // 43 | // Returning true from the callback will continue the traversal. 44 | // Returning false will terminate the traversal. 45 | // 46 | // A non-nil error will abort the traversal, and the error will be 47 | // propagated to the caller. 48 | ForEach(func(shard.Key) (bool, error)) error 49 | 50 | // Size returns the size of the repo in bytes. 51 | Size() (uint64, error) 52 | } 53 | 54 | // TODO unimplemented. 55 | type ManifestRepo interface { 56 | // ListManifests returns the available manifests for a given shard, 57 | // identified by their ManifestKey. 58 | ListManifests(key shard.Key) ([]ManifestKey, error) 59 | 60 | // GetManifest returns the Manifest identified by a given ManifestKey. 61 | GetManifest(key ManifestKey) (Manifest, error) 62 | 63 | // AddManifest adds a Manifest to the ManifestRepo. 64 | AddManifest(key ManifestKey, manifest Manifest) error 65 | 66 | // DropManifest drops a Manifest from the ManifestRepo. 67 | DropManifest(key ManifestKey) (bool, error) 68 | 69 | // StatManifest stats a Manifest. 70 | StatManifest(key ManifestKey) (Stat, error) 71 | } 72 | -------------------------------------------------------------------------------- /index/repo_fs.go: -------------------------------------------------------------------------------- 1 | package index 2 | 3 | import ( 4 | "fmt" 5 | "os" 6 | "path/filepath" 7 | "strings" 8 | 9 | "github.com/filecoin-project/dagstore/shard" 10 | carindex "github.com/ipld/go-car/v2/index" 11 | 12 | "golang.org/x/xerrors" 13 | ) 14 | 15 | const ( 16 | repoVersion = "1" 17 | indexSuffix = ".full.idx" 18 | ) 19 | 20 | // FSIndexRepo implements FullIndexRepo using the local file system to store 21 | // the indices 22 | type FSIndexRepo struct { 23 | baseDir string 24 | } 25 | 26 | var _ FullIndexRepo = (*FSIndexRepo)(nil) 27 | 28 | // NewFSRepo creates a new index repo that stores indices on the local 29 | // filesystem with the given base directory as the root 30 | func NewFSRepo(baseDir string) (*FSIndexRepo, error) { 31 | err := os.MkdirAll(baseDir, os.ModePerm) 32 | if err != nil { 33 | return nil, fmt.Errorf("failed to create index repo dir: %w", err) 34 | } 35 | 36 | l := &FSIndexRepo{baseDir: baseDir} 37 | 38 | // Get the repo version 39 | bs, err := os.ReadFile(l.versionPath()) 40 | if err != nil { 41 | // If the repo has not been initialized, write out the repo version file 42 | if os.IsNotExist(err) { 43 | err = os.WriteFile(l.versionPath(), []byte(repoVersion), 0666) 44 | if err != nil { 45 | return nil, err 46 | } 47 | return l, nil 48 | } 49 | 50 | // There was some other error 51 | return nil, err 52 | } 53 | 54 | // Check that this library can read this repo 55 | if string(bs) != repoVersion { 56 | return nil, xerrors.Errorf("cannot read existing repo with version %s", bs) 57 | } 58 | 59 | return l, nil 60 | } 61 | 62 | func (l *FSIndexRepo) GetFullIndex(key shard.Key) (carindex.Index, error) { 63 | path := l.indexPath(key) 64 | 65 | f, err := os.Open(path) 66 | if err != nil { 67 | return nil, err 68 | } 69 | 70 | defer f.Close() 71 | 72 | return carindex.ReadFrom(f) 73 | } 74 | 75 | func (l *FSIndexRepo) AddFullIndex(key shard.Key, index carindex.Index) (err error) { 76 | // Create a file at the key path 77 | f, err := os.Create(l.indexPath(key)) 78 | if err != nil { 79 | return err 80 | } 81 | defer f.Close() 82 | 83 | // Write the index to the file 84 | _, err = carindex.WriteTo(index, f) 85 | return err 86 | } 87 | 88 | func (l *FSIndexRepo) DropFullIndex(key shard.Key) (dropped bool, err error) { 89 | // Remove the file at the key path 90 | return true, os.Remove(l.indexPath(key)) 91 | } 92 | 93 | func (l *FSIndexRepo) StatFullIndex(key shard.Key) (Stat, error) { 94 | // Stat the file at the key path 95 | info, err := os.Stat(l.indexPath(key)) 96 | if err != nil { 97 | // Check if the file exists 98 | if os.IsNotExist(err) { 99 | // Should we return ErrNotFound instead of Stat{Exists:false} ? 100 | return Stat{Exists: false}, nil 101 | } 102 | return Stat{}, err 103 | } 104 | 105 | return Stat{ 106 | Exists: true, 107 | Size: uint64(info.Size()), 108 | }, nil 109 | } 110 | 111 | var stopWalk = xerrors.New("stop walk") 112 | 113 | // ForEach iterates over each index file to extract the key 114 | func (l *FSIndexRepo) ForEach(f func(shard.Key) (bool, error)) error { 115 | // Iterate over each index file 116 | err := l.eachIndexFile(func(info os.FileInfo) error { 117 | // The file name is derived by base 58 encoding the key 118 | // so decode the file name to get the key 119 | name := info.Name() 120 | name = name[:len(name)-len(indexSuffix)] 121 | k := shard.KeyFromString(name) 122 | 123 | // Call the callback with the key 124 | ok, err := f(k) 125 | if err != nil { 126 | return err 127 | } 128 | if !ok { 129 | return stopWalk 130 | } 131 | return nil 132 | }) 133 | if err == stopWalk { 134 | return nil 135 | } 136 | return err 137 | } 138 | 139 | // Len counts all index files in the base path 140 | func (l *FSIndexRepo) Len() (int, error) { 141 | ret := 0 142 | err := l.eachIndexFile(func(info os.FileInfo) error { 143 | ret++ 144 | return nil 145 | }) 146 | return ret, err 147 | } 148 | 149 | // Size sums the size of all index files in the base path 150 | func (l *FSIndexRepo) Size() (uint64, error) { 151 | var size uint64 152 | err := l.eachIndexFile(func(info os.FileInfo) error { 153 | size += uint64(info.Size()) 154 | return nil 155 | }) 156 | return size, err 157 | } 158 | 159 | // eachIndexFile calls the callback for each index file 160 | func (l *FSIndexRepo) eachIndexFile(f func(info os.FileInfo) error) error { 161 | return filepath.Walk(l.baseDir, func(path string, info os.FileInfo, err error) error { 162 | if strings.HasSuffix(info.Name(), indexSuffix) { 163 | return f(info) 164 | } 165 | return nil 166 | }) 167 | } 168 | 169 | func (l *FSIndexRepo) indexPath(key shard.Key) string { 170 | return filepath.Join(l.baseDir, key.String()+indexSuffix) 171 | } 172 | 173 | func (l *FSIndexRepo) versionPath() string { 174 | return filepath.Join(l.baseDir, ".version") 175 | } 176 | -------------------------------------------------------------------------------- /index/repo_fs_test.go: -------------------------------------------------------------------------------- 1 | package index 2 | 3 | import ( 4 | "os" 5 | "testing" 6 | 7 | "github.com/filecoin-project/dagstore/shard" 8 | "github.com/ipfs/go-cid" 9 | carindex "github.com/ipld/go-car/v2/index" 10 | "github.com/multiformats/go-multicodec" 11 | "github.com/stretchr/testify/require" 12 | "github.com/stretchr/testify/suite" 13 | ) 14 | 15 | func TestFSRepo(t *testing.T) { 16 | basePath := t.TempDir() 17 | repo, err := NewFSRepo(basePath) 18 | require.NoError(t, err) 19 | 20 | suite.Run(t, &fullIndexRepoSuite{impl: repo}) 21 | } 22 | 23 | func TestFSRepoVersions(t *testing.T) { 24 | basePath := t.TempDir() 25 | repo, err := NewFSRepo(basePath) 26 | require.NoError(t, err) 27 | 28 | // Expect the repo to have been initialized with the correct version 29 | bs, err := os.ReadFile(repo.versionPath()) 30 | require.Equal(t, repoVersion, string(bs)) 31 | 32 | // Verify we can create a new repo at the same path 33 | _, err = NewFSRepo(basePath) 34 | require.NoError(t, err) 35 | 36 | // Verify that creating a repo at a path with a higher different version 37 | // returns an error 38 | err = os.WriteFile(repo.versionPath(), []byte("2"), 0666) 39 | _, err = NewFSRepo(basePath) 40 | require.Error(t, err) 41 | } 42 | 43 | func TestFSRepoLoadFromDisk(t *testing.T) { 44 | basePath := t.TempDir() 45 | 46 | cid1, err := cid.Parse("bafykbzaceaeqhm77anl5mv2wjkmh4ofyf6s6eww3ujfmhtsfab65vi3rlccaq") 47 | require.NoError(t, err) 48 | offset1 := uint64(10) 49 | k := shard.KeyFromString("shard-key-1") 50 | 51 | // make an index 52 | idx, err := carindex.New(multicodec.CarIndexSorted) 53 | require.NoError(t, err) 54 | err = idx.Load([]carindex.Record{{Cid: cid1, Offset: offset1}}) 55 | require.NoError(t, err) 56 | 57 | // Create a repo at the base path 58 | repo1, err := NewFSRepo(basePath) 59 | require.NoError(t, err) 60 | 61 | // Add an index to the repo 62 | err = repo1.AddFullIndex(k, idx) 63 | require.NoError(t, err) 64 | 65 | // Create a new repo at the same path 66 | repo2, err := NewFSRepo(basePath) 67 | require.NoError(t, err) 68 | 69 | // Verify that we can get the index from the repo and do a lookup 70 | fidx, err := repo2.GetFullIndex(k) 71 | require.NoError(t, err) 72 | 73 | offset, err := carindex.GetFirst(fidx, cid1) 74 | require.NoError(t, err) 75 | require.Equal(t, offset1, offset) 76 | } 77 | -------------------------------------------------------------------------------- /index/repo_mem.go: -------------------------------------------------------------------------------- 1 | package index 2 | 3 | import ( 4 | "bytes" 5 | "sync" 6 | 7 | "github.com/filecoin-project/dagstore/shard" 8 | "github.com/ipld/go-car/v2/index" 9 | ) 10 | 11 | // MemIndexRepo implements FullIndexRepo with an in-memory map. 12 | type MemIndexRepo struct { 13 | lk sync.RWMutex 14 | idxs map[shard.Key]index.Index 15 | } 16 | 17 | func NewMemoryRepo() *MemIndexRepo { 18 | return &MemIndexRepo{idxs: make(map[shard.Key]index.Index)} 19 | } 20 | 21 | func (m *MemIndexRepo) GetFullIndex(key shard.Key) (idx index.Index, err error) { 22 | m.lk.RLock() 23 | defer m.lk.RUnlock() 24 | 25 | idx, ok := m.idxs[key] 26 | if !ok { 27 | return nil, ErrNotFound 28 | } 29 | return idx, nil 30 | } 31 | 32 | func (m *MemIndexRepo) AddFullIndex(key shard.Key, index index.Index) (err error) { 33 | m.lk.Lock() 34 | defer m.lk.Unlock() 35 | 36 | m.idxs[key] = index 37 | 38 | return nil 39 | } 40 | 41 | func (m *MemIndexRepo) DropFullIndex(key shard.Key) (dropped bool, err error) { 42 | m.lk.Lock() 43 | defer m.lk.Unlock() 44 | 45 | // TODO need to check if the index exists to be able to report whether it was dropped or not. 46 | delete(m.idxs, key) 47 | 48 | return true, nil 49 | } 50 | 51 | func (m *MemIndexRepo) StatFullIndex(key shard.Key) (Stat, error) { 52 | m.lk.RLock() 53 | defer m.lk.RUnlock() 54 | 55 | _, ok := m.idxs[key] 56 | if !ok { 57 | return Stat{Exists: false}, nil 58 | } 59 | 60 | size, err := m.indexSize(key) 61 | if err != nil { 62 | return Stat{}, err 63 | } 64 | 65 | return Stat{ 66 | Exists: ok, 67 | Size: size, 68 | }, nil 69 | } 70 | 71 | func (m *MemIndexRepo) ForEach(f func(shard.Key) (bool, error)) error { 72 | m.lk.RLock() 73 | ks := make([]shard.Key, 0, len(m.idxs)) 74 | for k := range m.idxs { 75 | ks = append(ks, k) 76 | } 77 | m.lk.RUnlock() 78 | 79 | for _, k := range ks { 80 | ok, err := f(k) 81 | if err != nil { 82 | return err 83 | } 84 | if !ok { 85 | return nil 86 | } 87 | } 88 | return nil 89 | } 90 | 91 | func (m *MemIndexRepo) Len() (int, error) { 92 | m.lk.RLock() 93 | defer m.lk.RUnlock() 94 | 95 | return len(m.idxs), nil 96 | } 97 | 98 | func (m *MemIndexRepo) Size() (uint64, error) { 99 | m.lk.RLock() 100 | defer m.lk.RUnlock() 101 | 102 | var size uint64 103 | for k := range m.idxs { 104 | k := shard.Key(k) 105 | sz, err := m.indexSize(k) 106 | if err != nil { 107 | return 0, err 108 | } 109 | size += sz 110 | } 111 | return size, nil 112 | } 113 | 114 | func (m *MemIndexRepo) indexSize(k shard.Key) (uint64, error) { 115 | idx, ok := m.idxs[k] 116 | if !ok { 117 | return 0, ErrNotFound 118 | } 119 | 120 | // Marshal the index just to get the size. 121 | // Could optimize by memoizing this although I don't think it's necessary 122 | // as the memory index repo is likely only used in tests. 123 | var buff bytes.Buffer 124 | _, err := index.WriteTo(idx, &buff) 125 | if err != nil { 126 | return 0, err 127 | } 128 | return uint64(buff.Len()), nil 129 | } 130 | 131 | var _ FullIndexRepo = (*MemIndexRepo)(nil) 132 | -------------------------------------------------------------------------------- /index/repo_mem_test.go: -------------------------------------------------------------------------------- 1 | package index 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/suite" 7 | ) 8 | 9 | func TestMemIndexRepo(t *testing.T) { 10 | suite.Run(t, &fullIndexRepoSuite{impl: NewMemoryRepo()}) 11 | } 12 | -------------------------------------------------------------------------------- /index/repo_test.go: -------------------------------------------------------------------------------- 1 | package index 2 | 3 | import ( 4 | "bytes" 5 | 6 | carindex "github.com/ipld/go-car/v2/index" 7 | "github.com/multiformats/go-multicodec" 8 | "golang.org/x/xerrors" 9 | 10 | "github.com/filecoin-project/dagstore/shard" 11 | 12 | "github.com/ipfs/go-cid" 13 | 14 | "github.com/stretchr/testify/require" 15 | "github.com/stretchr/testify/suite" 16 | ) 17 | 18 | type fullIndexRepoSuite struct { 19 | suite.Suite 20 | impl FullIndexRepo 21 | } 22 | 23 | func (s *fullIndexRepoSuite) TestAllMethods() { 24 | r := s.impl 25 | t := s.T() 26 | 27 | cid1, err := cid.Parse("bafykbzaceaeqhm77anl5mv2wjkmh4ofyf6s6eww3ujfmhtsfab65vi3rlccaq") 28 | require.NoError(t, err) 29 | offset1 := uint64(10) 30 | k := shard.KeyFromString("shard-key-1") 31 | 32 | // make an index 33 | idx, err := carindex.New(multicodec.CarIndexSorted) 34 | require.NoError(t, err) 35 | err = idx.Load([]carindex.Record{{Cid: cid1, Offset: offset1}}) 36 | require.NoError(t, err) 37 | 38 | // Verify that an empty repo has zero size 39 | stat, err := r.StatFullIndex(k) 40 | require.NoError(t, err) 41 | require.False(t, stat.Exists) 42 | require.EqualValues(t, 0, stat.Size) 43 | 44 | l, err := r.Len() 45 | require.EqualValues(t, 0, l) 46 | 47 | size, err := r.Size() 48 | require.EqualValues(t, 0, size) 49 | 50 | // Verify that there is an error trying to retrieve an index before it's added 51 | _, err = r.GetFullIndex(k) 52 | require.Error(t, err) 53 | 54 | // Add an index 55 | err = r.AddFullIndex(k, idx) 56 | require.NoError(t, err) 57 | 58 | l, err = r.Len() 59 | require.EqualValues(t, 1, l) 60 | 61 | // Verify the size of the index is correct 62 | var b bytes.Buffer 63 | _, err = carindex.WriteTo(idx, &b) 64 | require.NoError(t, err) 65 | expStatSize := b.Len() 66 | 67 | stat, err = r.StatFullIndex(k) 68 | require.NoError(t, err) 69 | require.True(t, stat.Exists) 70 | require.EqualValues(t, expStatSize, stat.Size) 71 | 72 | size, err = r.Size() 73 | require.EqualValues(t, expStatSize, size) 74 | 75 | count := 0 76 | err = r.ForEach(func(key shard.Key) (bool, error) { 77 | if key != k { 78 | return false, xerrors.Errorf("for each returned wrong key") 79 | } 80 | count++ 81 | return true, nil 82 | }) 83 | require.NoError(t, err) 84 | require.Equal(t, 1, count) 85 | 86 | // Verify that we can retrieve an index and perform a lookup 87 | fidx, err := r.GetFullIndex(k) 88 | require.NoError(t, err) 89 | 90 | offset, err := carindex.GetFirst(fidx, cid1) 91 | require.NoError(t, err) 92 | require.Equal(t, offset1, offset) 93 | 94 | // Drop the index 95 | dropped, err := r.DropFullIndex(k) 96 | require.NoError(t, err) 97 | require.True(t, dropped) 98 | 99 | // Verify that the index is no longer present 100 | stat, err = r.StatFullIndex(k) 101 | require.NoError(t, err) 102 | require.False(t, stat.Exists) 103 | require.EqualValues(t, 0, stat.Size) 104 | 105 | l, err = r.Len() 106 | require.EqualValues(t, 0, l) 107 | 108 | size, err = r.Size() 109 | require.EqualValues(t, 0, size) 110 | } 111 | -------------------------------------------------------------------------------- /indexbs/indexbacked_bs.go: -------------------------------------------------------------------------------- 1 | package indexbs 2 | 3 | import ( 4 | "context" 5 | "errors" 6 | "fmt" 7 | "sync" 8 | "time" 9 | 10 | "github.com/filecoin-project/dagstore" 11 | "github.com/filecoin-project/dagstore/shard" 12 | blocks "github.com/ipfs/go-block-format" 13 | "github.com/ipfs/go-cid" 14 | "github.com/ipfs/go-datastore" 15 | blockstore "github.com/ipfs/go-ipfs-blockstore" 16 | logging "github.com/ipfs/go-log/v2" 17 | "github.com/jellydator/ttlcache/v2" 18 | ) 19 | 20 | var logbs = logging.Logger("dagstore/idxbs") 21 | 22 | var ErrBlockNotFound = errors.New("block not found") 23 | 24 | var _ blockstore.Blockstore = (*IndexBackedBlockstore)(nil) 25 | 26 | // ErrNoShardSelected means that the shard selection function rejected all of the given shards. 27 | var ErrNoShardSelected = errors.New("no shard selected") 28 | 29 | // ShardSelectorF helps select a shard to fetch a cid from if the given cid is present in multiple shards. 30 | // It should return `ErrNoShardSelected` if none of the given shard is selected. 31 | type ShardSelectorF func(c cid.Cid, shards []shard.Key) (shard.Key, error) 32 | 33 | type accessorWithBlockstore struct { 34 | sa *dagstore.ShardAccessor 35 | bs dagstore.ReadBlockstore 36 | } 37 | 38 | type IdxBstoreDagstore interface { 39 | ShardsContainingCid(ctx context.Context, c cid.Cid) ([]shard.Key, error) 40 | AcquireShard(ctx context.Context, key shard.Key, out chan dagstore.ShardResult, _ dagstore.AcquireOpts) error 41 | } 42 | 43 | // IndexBackedBlockstore is a read only blockstore over all cids across all shards in the dagstore. 44 | type IndexBackedBlockstore struct { 45 | ctx context.Context 46 | d IdxBstoreDagstore 47 | shardSelectF ShardSelectorF 48 | 49 | // caches the blockstore for a given shard for shard read affinity 50 | // i.e. further reads will likely be from the same shard. Maps (shard key -> blockstore). 51 | blockstoreCache *ttlcache.Cache 52 | // used to manage concurrent acquisition of shards by multiple threads 53 | stripedLock [256]sync.Mutex 54 | } 55 | 56 | func NewIndexBackedBlockstore(ctx context.Context, d IdxBstoreDagstore, shardSelector ShardSelectorF, maxCacheSize int, cacheExpire time.Duration) (blockstore.Blockstore, error) { 57 | cache := ttlcache.NewCache() 58 | cache.SetTTL(cacheExpire) 59 | cache.SetCacheSizeLimit(maxCacheSize) 60 | cache.SetExpirationReasonCallback(func(_ string, _ ttlcache.EvictionReason, val interface{}) { 61 | // Ensure we close the blockstore for a shard when it's evicted from 62 | // the cache so dagstore can gc it. 63 | // TODO: add reference counting mechanism so that the blockstore does 64 | // not get closed while there is an operation still in progress against it 65 | abs := val.(*accessorWithBlockstore) 66 | abs.sa.Close() 67 | }) 68 | 69 | return &IndexBackedBlockstore{ 70 | ctx: ctx, 71 | d: d, 72 | shardSelectF: shardSelector, 73 | blockstoreCache: cache, 74 | }, nil 75 | } 76 | 77 | type BlockstoreOp bool 78 | 79 | const ( 80 | BlockstoreOpGet = true 81 | BlockstoreOpGetSize = !BlockstoreOpGet 82 | ) 83 | 84 | func (o BlockstoreOp) String() string { 85 | if o == BlockstoreOpGet { 86 | return "Get" 87 | } 88 | return "GetSize" 89 | } 90 | 91 | type opRes struct { 92 | block blocks.Block 93 | size int 94 | } 95 | 96 | func (ro *IndexBackedBlockstore) Get(ctx context.Context, c cid.Cid) (blocks.Block, error) { 97 | res, err := ro.execOpWithLogs(ctx, c, BlockstoreOpGet) 98 | if err != nil { 99 | return nil, err 100 | } 101 | return res.block, err 102 | } 103 | 104 | func (ro *IndexBackedBlockstore) GetSize(ctx context.Context, c cid.Cid) (int, error) { 105 | res, err := ro.execOpWithLogs(ctx, c, BlockstoreOpGetSize) 106 | if err != nil { 107 | return 0, err 108 | } 109 | return res.size, err 110 | } 111 | 112 | func (ro *IndexBackedBlockstore) execOpWithLogs(ctx context.Context, c cid.Cid, op BlockstoreOp) (*opRes, error) { 113 | logbs.Debugw(op.String(), "cid", c) 114 | 115 | res, err := ro.execOp(ctx, c, op) 116 | if err != nil { 117 | logbs.Debugw(op.String()+" error", "cid", c, "error", err) 118 | } else { 119 | logbs.Debugw(op.String()+" success", "cid", c) 120 | } 121 | return res, err 122 | } 123 | 124 | func (ro *IndexBackedBlockstore) execOp(ctx context.Context, c cid.Cid, op BlockstoreOp) (*opRes, error) { 125 | // Fetch all the shards containing the multihash 126 | shards, err := ro.d.ShardsContainingCid(ctx, c) 127 | if err != nil { 128 | if errors.Is(err, datastore.ErrNotFound) { 129 | return nil, ErrBlockNotFound 130 | } 131 | return nil, fmt.Errorf("failed to fetch shards containing block %s (multihash %s): %w", c, c.Hash(), err) 132 | } 133 | if len(shards) == 0 { 134 | // If there are no shards containing the multihash, return "not found" 135 | return nil, ErrBlockNotFound 136 | } 137 | 138 | // Do we have a cached blockstore for a shard containing the required block? 139 | // If so, call op on the cached blockstore. 140 | for _, sk := range shards { 141 | // Get the shard's blockstore from the cache 142 | val, err := ro.blockstoreCache.Get(sk.String()) 143 | if err != nil { 144 | continue 145 | } 146 | 147 | if val != nil { 148 | accessor := val.(*accessorWithBlockstore) 149 | res, err := execOpOnBlockstore(ctx, c, sk, accessor.bs, op) 150 | if err != nil { 151 | return nil, err 152 | } 153 | return res, nil 154 | } 155 | } 156 | 157 | // We weren't able to find a cached blockstore for a shard that contains 158 | // the block. Create a new blockstore for the shard. 159 | 160 | // Use the shard select function to select one of the shards with the block 161 | sk, err := ro.shardSelectF(c, shards) 162 | if err != nil && errors.Is(err, ErrNoShardSelected) { 163 | // If none of the shards passes the selection filter, return "not found" 164 | return nil, ErrBlockNotFound 165 | } 166 | if err != nil { 167 | return nil, fmt.Errorf("failed to run shard selection function: %w", err) 168 | } 169 | 170 | // Some retrieval patterns will result in multiple threads fetching blocks 171 | // from the same piece concurrently. In that case many threads may attempt 172 | // to create a blockstore over the same piece. Use a striped lock to ensure 173 | // that the blockstore is only created once for all threads waiting on the 174 | // same shard. 175 | bs, err := func() (dagstore.ReadBlockstore, error) { 176 | // Derive the striped lock index from the shard key and acquire the lock 177 | skstr := sk.String() 178 | lockIdx := skstr[len(skstr)-1] 179 | ro.stripedLock[lockIdx].Lock() 180 | defer ro.stripedLock[lockIdx].Unlock() 181 | 182 | // Check if the blockstore was created by another thread while this 183 | // thread was waiting to enter the lock 184 | val, err := ro.blockstoreCache.Get(sk.String()) 185 | if err == nil && val != nil { 186 | return val.(*accessorWithBlockstore).bs, nil 187 | } 188 | 189 | // Acquire the blockstore for the selected shard 190 | resch := make(chan dagstore.ShardResult, 1) 191 | if err := ro.d.AcquireShard(ro.ctx, sk, resch, dagstore.AcquireOpts{}); err != nil { 192 | return nil, fmt.Errorf("failed to acquire shard %s: %w", sk, err) 193 | } 194 | var shres dagstore.ShardResult 195 | select { 196 | case <-ctx.Done(): 197 | return nil, ctx.Err() 198 | case shres = <-resch: 199 | if shres.Error != nil { 200 | return nil, fmt.Errorf("failed to acquire shard %s: %w", sk, shres.Error) 201 | } 202 | } 203 | 204 | sa := shres.Accessor 205 | bs, err := sa.Blockstore() 206 | if err != nil { 207 | return nil, fmt.Errorf("failed to load read-only blockstore for shard %s: %w", sk, err) 208 | } 209 | 210 | // Add the blockstore to the cache 211 | ro.blockstoreCache.Set(sk.String(), &accessorWithBlockstore{sa, bs}) 212 | 213 | logbs.Debugw("Added new blockstore to cache", "cid", c, "shard", sk) 214 | 215 | return bs, nil 216 | }() 217 | if err != nil { 218 | return nil, err 219 | } 220 | 221 | // Call the operation on the blockstore 222 | return execOpOnBlockstore(ctx, c, sk, bs, op) 223 | } 224 | 225 | func execOpOnBlockstore(ctx context.Context, c cid.Cid, sk shard.Key, bs dagstore.ReadBlockstore, op BlockstoreOp) (*opRes, error) { 226 | var err error 227 | var res opRes 228 | switch op { 229 | case BlockstoreOpGet: 230 | res.block, err = bs.Get(ctx, c) 231 | case BlockstoreOpGetSize: 232 | res.size, err = bs.GetSize(ctx, c) 233 | } 234 | if err != nil { 235 | return nil, fmt.Errorf("failed to call blockstore.%s for shard %s: %w", op, sk, err) 236 | } 237 | return &res, nil 238 | } 239 | 240 | func (ro *IndexBackedBlockstore) Has(ctx context.Context, c cid.Cid) (bool, error) { 241 | logbs.Debugw("Has", "cid", c) 242 | 243 | // Get shards that contain the cid's hash 244 | shards, err := ro.d.ShardsContainingCid(ctx, c) 245 | if err != nil { 246 | logbs.Debugw("Has error", "cid", c, "err", err) 247 | return false, nil 248 | } 249 | if len(shards) == 0 { 250 | logbs.Debugw("Has: returning false", "cid", c) 251 | return false, nil 252 | } 253 | 254 | // Check if there is a shard with the block that is not filtered out by 255 | // the shard selection function 256 | _, err = ro.shardSelectF(c, shards) 257 | if err != nil && errors.Is(err, ErrNoShardSelected) { 258 | logbs.Debugw("Has: returning false", "cid", c) 259 | return false, nil 260 | } 261 | if err != nil { 262 | err = fmt.Errorf("failed to run shard selection function: %w", err) 263 | logbs.Debugw("Has error", "cid", c, "err", err) 264 | return false, err 265 | } 266 | 267 | logbs.Debugw("Has: returning true", "cid", c) 268 | return true, nil 269 | } 270 | 271 | // --- UNSUPPORTED BLOCKSTORE METHODS ------- 272 | func (ro *IndexBackedBlockstore) DeleteBlock(context.Context, cid.Cid) error { 273 | return errors.New("unsupported operation DeleteBlock") 274 | } 275 | func (ro *IndexBackedBlockstore) HashOnRead(_ bool) {} 276 | func (ro *IndexBackedBlockstore) Put(context.Context, blocks.Block) error { 277 | return errors.New("unsupported operation Put") 278 | } 279 | func (ro *IndexBackedBlockstore) PutMany(context.Context, []blocks.Block) error { 280 | return errors.New("unsupported operation PutMany") 281 | } 282 | func (ro *IndexBackedBlockstore) AllKeysChan(ctx context.Context) (<-chan cid.Cid, error) { 283 | return nil, errors.New("unsupported operation AllKeysChan") 284 | } 285 | 286 | type IdxBstoreDagstoreFromDagstore struct { 287 | dagstore.Interface 288 | } 289 | 290 | var _ IdxBstoreDagstore = (*IdxBstoreDagstoreFromDagstore)(nil) 291 | 292 | func (d *IdxBstoreDagstoreFromDagstore) ShardsContainingCid(ctx context.Context, c cid.Cid) ([]shard.Key, error) { 293 | return d.Interface.ShardsContainingMultihash(ctx, c.Hash()) 294 | } 295 | -------------------------------------------------------------------------------- /indexbs/indexbacked_bs_test.go: -------------------------------------------------------------------------------- 1 | package indexbs 2 | 3 | import ( 4 | "context" 5 | "errors" 6 | "fmt" 7 | "io/ioutil" 8 | "testing" 9 | "time" 10 | 11 | "golang.org/x/sync/errgroup" 12 | 13 | "github.com/filecoin-project/dagstore" 14 | "github.com/filecoin-project/dagstore/mount" 15 | "github.com/filecoin-project/dagstore/testdata" 16 | 17 | "github.com/multiformats/go-multihash" 18 | 19 | "github.com/filecoin-project/dagstore/shard" 20 | "github.com/ipfs/go-cid" 21 | "github.com/ipfs/go-datastore" 22 | dssync "github.com/ipfs/go-datastore/sync" 23 | "github.com/stretchr/testify/require" 24 | ) 25 | 26 | var noOpSelector = func(c cid.Cid, shards []shard.Key) (shard.Key, error) { 27 | return shards[0], nil 28 | } 29 | 30 | var carv2mnt = &mount.FSMount{FS: testdata.FS, Path: testdata.FSPathCarV2} 31 | 32 | func TestIndexBackedBlockstore(t *testing.T) { 33 | ctx := context.Background() 34 | store := dssync.MutexWrap(datastore.NewMapDatastore()) 35 | dagst, err := dagstore.NewDAGStore(dagstore.Config{ 36 | MountRegistry: testRegistry(t), 37 | TransientsDir: t.TempDir(), 38 | Datastore: store, 39 | }) 40 | require.NoError(t, err) 41 | 42 | err = dagst.Start(context.Background()) 43 | require.NoError(t, err) 44 | 45 | // register a shard 46 | ch := make(chan dagstore.ShardResult, 1) 47 | sk := shard.KeyFromString("test1") 48 | err = dagst.RegisterShard(context.Background(), sk, carv2mnt, ch, dagstore.RegisterOpts{}) 49 | require.NoError(t, err) 50 | res := <-ch 51 | require.NoError(t, res.Error) 52 | 53 | ibsapi := &IdxBstoreDagstoreFromDagstore{Interface: dagst} 54 | rbs, err := NewIndexBackedBlockstore(ctx, ibsapi, noOpSelector, 10, time.Minute) 55 | require.NoError(t, err) 56 | 57 | // iterate over the CARV2 Index for the given CARv2 file and ensure the readonly blockstore 58 | // works for each of those cids 59 | it, err := dagst.GetIterableIndex(sk) 60 | require.NoError(t, err) 61 | 62 | var errg errgroup.Group 63 | 64 | it.ForEach(func(mh multihash.Multihash, _ uint64) error { 65 | 66 | mhs := mh 67 | errg.Go(func() error { 68 | c := cid.NewCidV1(cid.Raw, mhs) 69 | 70 | // Has 71 | has, err := rbs.Has(ctx, c) 72 | if err != nil { 73 | return err 74 | } 75 | if !has { 76 | return errors.New("has should be true") 77 | } 78 | 79 | // Get 80 | blk, err := rbs.Get(ctx, c) 81 | if err != nil { 82 | return err 83 | } 84 | if blk == nil { 85 | return errors.New("block should not be empty") 86 | } 87 | 88 | // GetSize 89 | _, err = rbs.GetSize(ctx, c) 90 | if err != nil { 91 | return err 92 | } 93 | 94 | // ensure cids match 95 | if blk.Cid() != c { 96 | return errors.New("cid mismatch") 97 | } 98 | return nil 99 | 100 | }) 101 | 102 | return nil 103 | }) 104 | 105 | require.NoError(t, errg.Wait()) 106 | 107 | // ------------------------------------------ 108 | // Test with a shard selector that returns an error and verify all methods 109 | // return the error 110 | rejectedErr := errors.New("rejected") 111 | fss := func(c cid.Cid, shards []shard.Key) (shard.Key, error) { 112 | return shard.Key{}, rejectedErr 113 | } 114 | 115 | rbs, err = NewIndexBackedBlockstore(ctx, ibsapi, fss, 10, time.Minute) 116 | require.NoError(t, err) 117 | it.ForEach(func(mh multihash.Multihash, u uint64) error { 118 | c := cid.NewCidV1(cid.Raw, mh) 119 | 120 | has, err := rbs.Has(ctx, c) 121 | require.ErrorIs(t, err, rejectedErr) 122 | require.False(t, has) 123 | 124 | blk, err := rbs.Get(ctx, c) 125 | require.ErrorIs(t, err, rejectedErr) 126 | require.Empty(t, blk) 127 | 128 | sz, err := rbs.GetSize(ctx, c) 129 | require.ErrorIs(t, err, rejectedErr) 130 | require.EqualValues(t, 0, sz) 131 | 132 | return nil 133 | }) 134 | 135 | // ------------------------------------------ 136 | // Test with a shard selector that returns ErrNoShardSelected 137 | fss = func(c cid.Cid, shards []shard.Key) (shard.Key, error) { 138 | return shard.Key{}, ErrNoShardSelected 139 | } 140 | 141 | rbs, err = NewIndexBackedBlockstore(ctx, ibsapi, fss, 10, time.Minute) 142 | require.NoError(t, err) 143 | it.ForEach(func(mh multihash.Multihash, u uint64) error { 144 | c := cid.NewCidV1(cid.Raw, mh) 145 | 146 | // Has should return false 147 | has, err := rbs.Has(ctx, c) 148 | require.NoError(t, err) 149 | require.False(t, has) 150 | 151 | // Get should return ErrBlockNotFound 152 | blk, err := rbs.Get(ctx, c) 153 | require.ErrorIs(t, err, ErrBlockNotFound) 154 | require.Empty(t, blk) 155 | 156 | // GetSize should return ErrBlockNotFound 157 | sz, err := rbs.GetSize(ctx, c) 158 | require.ErrorIs(t, err, ErrBlockNotFound) 159 | require.EqualValues(t, 0, sz) 160 | 161 | return nil 162 | }) 163 | 164 | // ------------------------------------------ 165 | // Test with a cid that isn't in the shard 166 | notFoundCid, err := cid.Parse("bafzbeigai3eoy2ccc7ybwjfz5r3rdxqrinwi4rwytly24tdbh6yk7zslrm") 167 | require.NoError(t, err) 168 | 169 | rbs, err = NewIndexBackedBlockstore(ctx, ibsapi, noOpSelector, 10, time.Minute) 170 | require.NoError(t, err) 171 | 172 | // Has should return false 173 | has, err := rbs.Has(ctx, notFoundCid) 174 | require.NoError(t, err) 175 | require.False(t, has) 176 | 177 | // Get should return ErrBlockNotFound 178 | blk, err := rbs.Get(ctx, notFoundCid) 179 | require.ErrorIs(t, err, ErrBlockNotFound) 180 | require.Empty(t, blk) 181 | 182 | // GetSize should return ErrBlockNotFound 183 | sz, err := rbs.GetSize(ctx, notFoundCid) 184 | require.ErrorIs(t, err, ErrBlockNotFound) 185 | require.EqualValues(t, 0, sz) 186 | } 187 | 188 | func TestIndexBackedBlockstoreFuzz(t *testing.T) { 189 | ctx := context.Background() 190 | tempdir := t.TempDir() 191 | store := dssync.MutexWrap(datastore.NewMapDatastore()) 192 | dagst, err := dagstore.NewDAGStore(dagstore.Config{ 193 | MountRegistry: testRegistry(t), 194 | TransientsDir: t.TempDir(), 195 | Datastore: store, 196 | }) 197 | require.NoError(t, err) 198 | 199 | err = dagst.Start(context.Background()) 200 | require.NoError(t, err) 201 | 202 | // register some shards 203 | var sks []shard.Key 204 | for i := 0; i < 10; i++ { 205 | ch := make(chan dagstore.ShardResult, 1) 206 | sk := shard.KeyFromString(fmt.Sprintf("test%d", i)) 207 | 208 | rseed := time.Now().Nanosecond() 209 | randomFilepath, err := testdata.CreateRandomFile(tempdir, rseed, 256*1024) 210 | require.NoError(t, err) 211 | _, carFilepath, err := testdata.CreateDenseCARv2(tempdir, randomFilepath) 212 | require.NoError(t, err) 213 | carBytes, err := ioutil.ReadFile(carFilepath) 214 | require.NoError(t, err) 215 | mnt := &mount.BytesMount{Bytes: carBytes} 216 | err = dagst.RegisterShard(context.Background(), sk, mnt, ch, dagstore.RegisterOpts{}) 217 | require.NoError(t, err) 218 | res := <-ch 219 | require.NoError(t, res.Error) 220 | sks = append(sks, sk) 221 | } 222 | 223 | ibsapi := &IdxBstoreDagstoreFromDagstore{Interface: dagst} 224 | rbs, err := NewIndexBackedBlockstore(ctx, ibsapi, noOpSelector, 3, time.Minute) 225 | require.NoError(t, err) 226 | 227 | var errg errgroup.Group 228 | for _, sk := range sks { 229 | sk := sk 230 | errg.Go(func() error { 231 | it, err := dagst.GetIterableIndex(sk) 232 | if err != nil { 233 | return err 234 | } 235 | 236 | for i := 0; i < 10; i++ { 237 | var skerrg errgroup.Group 238 | it.ForEach(func(mh multihash.Multihash, _ uint64) error { 239 | mhs := mh 240 | c := cid.NewCidV1(cid.Raw, mhs) 241 | skerrg.Go(func() error { 242 | has, err := rbs.Has(ctx, c) 243 | if err != nil { 244 | return err 245 | } 246 | if !has { 247 | return errors.New("has should be true") 248 | } 249 | return nil 250 | }) 251 | 252 | skerrg.Go(func() error { 253 | blk, err := rbs.Get(ctx, c) 254 | if err != nil { 255 | return err 256 | } 257 | if blk == nil { 258 | return errors.New("block should not be empty") 259 | } 260 | 261 | // ensure cids match 262 | if blk.Cid() != c { 263 | return errors.New("cid mismatch") 264 | } 265 | return nil 266 | }) 267 | 268 | skerrg.Go(func() error { 269 | _, err := rbs.GetSize(ctx, c) 270 | return err 271 | }) 272 | 273 | return nil 274 | }) 275 | err := skerrg.Wait() 276 | if err != nil { 277 | return err 278 | } 279 | } 280 | return nil 281 | }) 282 | } 283 | require.NoError(t, errg.Wait()) 284 | } 285 | 286 | func testRegistry(t *testing.T) *mount.Registry { 287 | r := mount.NewRegistry() 288 | err := r.Register("fs", &mount.FSMount{FS: testdata.FS}) 289 | require.NoError(t, err) 290 | err = r.Register("counting", new(mount.Counting)) 291 | require.NoError(t, err) 292 | return r 293 | } 294 | -------------------------------------------------------------------------------- /interface.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | import ( 4 | "context" 5 | 6 | carindex "github.com/ipld/go-car/v2/index" 7 | mh "github.com/multiformats/go-multihash" 8 | 9 | "github.com/filecoin-project/dagstore/mount" 10 | "github.com/filecoin-project/dagstore/shard" 11 | ) 12 | 13 | // Interface is the publicly exposed interface of the DAGStore. It exists 14 | // for mocking or DI purposes. 15 | type Interface interface { 16 | Start(ctx context.Context) error 17 | RegisterShard(ctx context.Context, key shard.Key, mnt mount.Mount, out chan ShardResult, opts RegisterOpts) error 18 | DestroyShard(ctx context.Context, key shard.Key, out chan ShardResult, _ DestroyOpts) error 19 | AcquireShard(ctx context.Context, key shard.Key, out chan ShardResult, _ AcquireOpts) error 20 | RecoverShard(ctx context.Context, key shard.Key, out chan ShardResult, _ RecoverOpts) error 21 | GetShardInfo(k shard.Key) (ShardInfo, error) 22 | GetIterableIndex(key shard.Key) (carindex.IterableIndex, error) 23 | AllShardsInfo() AllShardsInfo 24 | ShardsContainingMultihash(ctx context.Context, h mh.Multihash) ([]shard.Key, error) 25 | GC(ctx context.Context) (*GCResult, error) 26 | Close() error 27 | } 28 | -------------------------------------------------------------------------------- /mount/bytes.go: -------------------------------------------------------------------------------- 1 | package mount 2 | 3 | import ( 4 | "bytes" 5 | "context" 6 | "encoding/base64" 7 | "net/url" 8 | ) 9 | 10 | // BytesMount encloses a byte slice. It is mainly used for testing. The 11 | // Upgrader passes through it. 12 | type BytesMount struct { 13 | Bytes []byte 14 | } 15 | 16 | var _ Mount = (*BytesMount)(nil) 17 | 18 | func (b *BytesMount) Fetch(_ context.Context) (Reader, error) { 19 | r := bytes.NewReader(b.Bytes) 20 | return &NopCloser{ 21 | Reader: r, 22 | ReaderAt: r, 23 | Seeker: r, 24 | }, nil 25 | } 26 | 27 | func (b *BytesMount) Info() Info { 28 | return Info{ 29 | Kind: KindLocal, 30 | AccessSequential: true, 31 | AccessSeek: true, 32 | AccessRandom: true, 33 | } 34 | } 35 | 36 | func (b *BytesMount) Stat(_ context.Context) (Stat, error) { 37 | return Stat{ 38 | Exists: true, 39 | Size: int64(len(b.Bytes)), 40 | }, nil 41 | } 42 | 43 | func (b *BytesMount) Serialize() *url.URL { 44 | return &url.URL{ 45 | Host: base64.StdEncoding.EncodeToString(b.Bytes), 46 | } 47 | } 48 | 49 | func (b *BytesMount) Deserialize(u *url.URL) error { 50 | decoded, err := base64.StdEncoding.DecodeString(u.Host) 51 | if err != nil { 52 | return err 53 | } 54 | b.Bytes = decoded 55 | return nil 56 | } 57 | 58 | func (b *BytesMount) Close() error { 59 | b.Bytes = nil // release 60 | return nil 61 | } 62 | -------------------------------------------------------------------------------- /mount/counting.go: -------------------------------------------------------------------------------- 1 | package mount 2 | 3 | import ( 4 | "context" 5 | "sync/atomic" 6 | ) 7 | 8 | // Counting is a mount that proxies to another mount and counts the number of 9 | // calls made to Fetch. It is mostly used in tests. 10 | type Counting struct { 11 | Mount 12 | 13 | n int32 14 | } 15 | 16 | func (c *Counting) Fetch(ctx context.Context) (Reader, error) { 17 | atomic.AddInt32(&c.n, 1) 18 | return c.Mount.Fetch(ctx) 19 | } 20 | 21 | func (c *Counting) Count() int { 22 | return int(atomic.LoadInt32(&c.n)) 23 | } 24 | -------------------------------------------------------------------------------- /mount/file.go: -------------------------------------------------------------------------------- 1 | package mount 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "net/url" 7 | "os" 8 | ) 9 | 10 | type FileMount struct { 11 | Path string 12 | } 13 | 14 | var _ Mount = (*FileMount)(nil) 15 | 16 | func (f *FileMount) Fetch(_ context.Context) (Reader, error) { 17 | return os.Open(f.Path) 18 | } 19 | 20 | func (f *FileMount) Info() Info { 21 | return Info{ 22 | Kind: KindLocal, 23 | AccessRandom: true, 24 | AccessSeek: true, 25 | AccessSequential: true, 26 | } 27 | } 28 | 29 | func (f *FileMount) Stat(_ context.Context) (Stat, error) { 30 | stat, err := os.Stat(f.Path) 31 | if err != nil && os.IsNotExist(err) { 32 | return Stat{}, err 33 | } 34 | return Stat{ 35 | Exists: !os.IsNotExist(err), 36 | Size: stat.Size(), 37 | }, err 38 | } 39 | 40 | func (f *FileMount) Serialize() *url.URL { 41 | return &url.URL{ 42 | Host: f.Path, 43 | } 44 | } 45 | 46 | func (f *FileMount) Deserialize(u *url.URL) error { 47 | if u.Host == "" { 48 | return fmt.Errorf("invalid host") 49 | } 50 | f.Path = u.Host 51 | return nil 52 | } 53 | 54 | func (f *FileMount) Close() error { 55 | return nil 56 | } 57 | -------------------------------------------------------------------------------- /mount/file_test.go: -------------------------------------------------------------------------------- 1 | package mount 2 | 3 | import ( 4 | "bytes" 5 | "context" 6 | "crypto/rand" 7 | "io" 8 | "io/ioutil" 9 | "testing" 10 | 11 | "github.com/stretchr/testify/require" 12 | ) 13 | 14 | func TestFileMount(t *testing.T) { 15 | const size = 1024 16 | 17 | // create a temp file. 18 | dir := t.TempDir() 19 | f, err := ioutil.TempFile(dir, "") 20 | require.NoError(t, err) 21 | defer f.Close() 22 | 23 | // read random junk into it up to 1kb; send a copy to a buffer 24 | // so we can compare. 25 | var b bytes.Buffer 26 | r := io.LimitReader(rand.Reader, size) 27 | r = io.TeeReader(r, &b) 28 | n, err := io.Copy(f, r) 29 | require.EqualValues(t, size, n) 30 | require.NoError(t, err) 31 | 32 | mnt := &FileMount{Path: f.Name()} 33 | stat, err := mnt.Stat(context.Background()) 34 | require.NoError(t, err) 35 | require.True(t, stat.Exists) 36 | require.EqualValues(t, size, stat.Size) 37 | 38 | // check URL. 39 | require.Equal(t, mnt.Path, mnt.Serialize().Host) 40 | 41 | info := mnt.Info() 42 | require.True(t, info.AccessSequential && info.AccessSeek && info.AccessRandom) // all flags true 43 | require.Equal(t, KindLocal, info.Kind) 44 | 45 | reader, err := mnt.Fetch(context.Background()) 46 | require.NoError(t, err) 47 | 48 | // sequential access 49 | read := make([]byte, 2000) 50 | i, err := reader.Read(read) 51 | require.NoError(t, err) 52 | require.EqualValues(t, size, i) // only truly read 1024 bytes. 53 | require.Equal(t, b.Bytes(), read[:i]) 54 | 55 | // seek to the beginning and read the first byte. 56 | n, err = reader.Seek(0, 0) 57 | require.NoError(t, err) 58 | var b1 [1]byte 59 | i, err = reader.Read(b1[:]) 60 | require.NoError(t, err) 61 | require.EqualValues(t, 1, i) 62 | require.Equal(t, b.Bytes()[0], b1[0]) 63 | 64 | // read a random byte. 65 | i, err = reader.ReadAt(b1[:], 100) 66 | require.NoError(t, err) 67 | require.EqualValues(t, 1, i) 68 | require.Equal(t, b.Bytes()[100], b1[0]) 69 | 70 | // close 71 | err = reader.Close() 72 | require.NoError(t, err) 73 | } 74 | -------------------------------------------------------------------------------- /mount/fs.go: -------------------------------------------------------------------------------- 1 | package mount 2 | 3 | import ( 4 | "context" 5 | "errors" 6 | "fmt" 7 | "io" 8 | "io/fs" 9 | "net/url" 10 | ) 11 | 12 | const path = "path" 13 | 14 | // FSMount is a mount that opens the file indicated by Path, using the 15 | // provided fs.FS. Given that io/fs does not support random access patterns, 16 | // this mount requires an Upgrade. It is suitable for testing. 17 | type FSMount struct { 18 | FS fs.FS 19 | Path string 20 | } 21 | 22 | var _ Mount = (*FSMount)(nil) 23 | 24 | func (f *FSMount) Close() error { 25 | return nil // TODO 26 | } 27 | 28 | func (f *FSMount) Fetch(ctx context.Context) (Reader, error) { 29 | // yield if the context is cancelled. 30 | select { 31 | case <-ctx.Done(): 32 | return nil, ctx.Err() 33 | default: 34 | } 35 | 36 | file, err := f.FS.Open(f.Path) 37 | if err != nil { 38 | return nil, err 39 | } 40 | ra, _ := file.(io.ReaderAt) 41 | sk, _ := file.(io.Seeker) 42 | return &fsReader{ 43 | File: file, 44 | ReaderAt: ra, 45 | Seeker: sk, 46 | }, err 47 | } 48 | 49 | func (f *FSMount) Info() Info { 50 | return Info{ 51 | Kind: KindLocal, 52 | AccessSequential: true, 53 | AccessSeek: false, // TODO actual capabilities depend on the fs.FS implementation! 54 | AccessRandom: false, // TODO actual capabilities depend on the fs.FS implementation! 55 | } 56 | } 57 | 58 | func (f *FSMount) Stat(_ context.Context) (Stat, error) { 59 | st, err := fs.Stat(f.FS, f.Path) 60 | if errors.Is(err, fs.ErrNotExist) { 61 | return Stat{Exists: false, Size: 0}, nil 62 | } 63 | if err != nil { 64 | return Stat{}, err 65 | } 66 | return Stat{ 67 | Exists: true, 68 | Size: st.Size(), 69 | }, nil 70 | } 71 | 72 | func (f *FSMount) Serialize() *url.URL { 73 | u := new(url.URL) 74 | if st, err := fs.Stat(f.FS, f.Path); err != nil { 75 | u.Host = "irrecoverable" 76 | } else { 77 | q := u.Query() 78 | q.Set(path, f.Path) 79 | u.RawQuery = q.Encode() 80 | u.Host = st.Name() 81 | } 82 | return u 83 | } 84 | 85 | func (f *FSMount) Deserialize(u *url.URL) error { 86 | if u.Host == "irrecoverable" || u.Host == "" { 87 | return fmt.Errorf("invalid host") 88 | } 89 | 90 | f.Path = u.Query().Get(path) 91 | return nil 92 | } 93 | 94 | type fsReader struct { 95 | fs.File 96 | io.ReaderAt 97 | io.Seeker 98 | } 99 | 100 | var _ Reader = (*fsReader)(nil) 101 | 102 | func (f *fsReader) ReadAt(p []byte, off int64) (n int, err error) { 103 | if f.ReaderAt == nil { 104 | return 0, ErrRandomAccessUnsupported 105 | } 106 | return f.ReaderAt.ReadAt(p, off) 107 | } 108 | 109 | func (f *fsReader) Seek(off int64, whence int) (int64, error) { 110 | if f.Seeker == nil { 111 | return 0, ErrSeekUnsupported 112 | } 113 | return f.Seeker.Seek(off, whence) 114 | } 115 | -------------------------------------------------------------------------------- /mount/mount.go: -------------------------------------------------------------------------------- 1 | package mount 2 | 3 | import ( 4 | "context" 5 | "errors" 6 | "io" 7 | "net/url" 8 | ) 9 | 10 | var ( 11 | // ErrSeekUnsupported is returned when Seek is called on a mount that is 12 | // not seekable. 13 | ErrSeekUnsupported = errors.New("mount does not support seek") 14 | 15 | // ErrRandomAccessUnsupported is returned when ReadAt is called on a mount 16 | // that does not support random access. 17 | ErrRandomAccessUnsupported = errors.New("mount does not support random access") 18 | ) 19 | 20 | // Kind is an enum describing the source of a Mount. 21 | type Kind int 22 | 23 | const ( 24 | // KindLocal indicates that the asset represented by this mount is of 25 | // transient provenance (e.g. filesystem mount). A call to Fetch() will open a 26 | // transient stream. 27 | // 28 | // Note that mounts of this kind may be indirectly backed by underlying storage 29 | // (e.g. NFS, FUSE), but from the viewpoint of the DAG store, the resource 30 | // is considered transient. 31 | KindLocal Kind = iota 32 | 33 | // KindRemote indicates that the asset represented by this mount is 34 | // fetched from a underlying provenance (e.g. HTTP, Filecoin sealing cluster, 35 | // IPFS, etc.) A call to Fetch() is likely to download the asset from 36 | // a underlying source, thus it is advisable to cache the asset locally once 37 | // downloaded. 38 | KindRemote 39 | ) 40 | 41 | // Mount is a pluggable component that represents the original location of the 42 | // data contained in a shard known to the DAG store. The backing resource is a 43 | // CAR file. 44 | // 45 | // Shards can be located anywhere, and can come and go dynamically e.g. Filecoin 46 | // deals expire, removable media is attached/detached, or the IPFS user purges 47 | // content. 48 | // 49 | // It is possible to mount shards with CARs accessible through the transient 50 | // filesystem, detachable mounts, NFS mounts, distributed filesystems like 51 | // Ceph/GlusterFS, HTTP, FTP, etc. 52 | // 53 | // Mount implementations are free to define constructor parameters or setters 54 | // to supply arguments needed to initialize the mount, such as credentials, 55 | // sector IDs, CIDs, etc. 56 | // 57 | // MountTypes must define a deterministic URL representation which will be used to: 58 | // 59 | // a. deserialise the Mount from DAG persistence when resuming the system by 60 | // using a pre-defined Mount factory mapped to the URL scheme. 61 | // b. support adding mounts from configuration files. 62 | type Mount interface { 63 | io.Closer 64 | 65 | // Fetch returns a Reader for this mount. Not all read access methods 66 | // may be supported. Check the Info object to determine which access methods 67 | // are effectively supported. 68 | // 69 | // To seamlessly upgrade a Mount to a fully-featured mount by using a transient 70 | // transient file, use the Upgrader. 71 | Fetch(ctx context.Context) (Reader, error) 72 | 73 | // Info describes the Mount. This is a pure function. 74 | Info() Info 75 | 76 | // Stat describes the underlying resource. 77 | Stat(ctx context.Context) (Stat, error) 78 | 79 | // Serialize returns a canonical URL that can be used to revive the Mount 80 | // after a restart. 81 | Serialize() *url.URL 82 | 83 | // Deserialize configures this Mount from the specified URL. 84 | Deserialize(*url.URL) error 85 | } 86 | 87 | // Reader is a fully-featured Reader returned from MountTypes. It is the 88 | // union of the standard IO sequential access method (Read), with seeking 89 | // ability (Seek), as well random access (ReadAt). 90 | type Reader interface { 91 | io.Closer 92 | io.Reader 93 | io.ReaderAt 94 | io.Seeker 95 | } 96 | 97 | // Info describes a mount. 98 | type Info struct { 99 | // Kind indicates the kind of mount. 100 | Kind Kind 101 | 102 | // TODO convert to bitfield 103 | AccessSequential bool 104 | AccessSeek bool 105 | AccessRandom bool 106 | } 107 | 108 | // Stat 109 | type Stat struct { 110 | // Exists indicates if the asset exists. 111 | Exists bool 112 | // Size is the size of the asset referred to by this Mount. 113 | Size int64 114 | // Ready indicates whether the mount can serve the resource immediately, or 115 | // if it needs to do work prior to serving it. 116 | Ready bool 117 | } 118 | 119 | type NopCloser struct { 120 | io.Reader 121 | io.ReaderAt 122 | io.Seeker 123 | } 124 | 125 | func (*NopCloser) Close() error { 126 | return nil 127 | } 128 | -------------------------------------------------------------------------------- /mount/registry.go: -------------------------------------------------------------------------------- 1 | package mount 2 | 3 | import ( 4 | "errors" 5 | "fmt" 6 | "net/url" 7 | "reflect" 8 | "sync" 9 | ) 10 | 11 | var ( 12 | // ErrUnrecognizedScheme is returned by Instantiate() when attempting to 13 | // initialize a Mount with an unrecognized URL scheme. 14 | ErrUnrecognizedScheme = errors.New("unrecognized mount scheme") 15 | 16 | // ErrUnrecognizedType is returned by Encode() when attempting to 17 | // represent a Mount whose type has not been registered. 18 | ErrUnrecognizedType = errors.New("unrecognized mount type") 19 | ) 20 | 21 | // Registry is a registry of Mount factories known to the DAG store. 22 | type Registry struct { 23 | lk sync.RWMutex 24 | byScheme map[string]Mount 25 | byType map[reflect.Type]string 26 | } 27 | 28 | // NewRegistry constructs a blank registry. 29 | func NewRegistry() *Registry { 30 | return &Registry{byScheme: map[string]Mount{}, byType: map[reflect.Type]string{}} 31 | } 32 | 33 | // Register adds a new mount type to the registry under the specified scheme. 34 | // 35 | // The supplied Mount is used as a template to create new instances. 36 | // 37 | // This means that the provided Mount can contain environmental configuration 38 | // that will be automatically carried over to all instances. 39 | func (r *Registry) Register(scheme string, template Mount) error { 40 | r.lk.Lock() 41 | defer r.lk.Unlock() 42 | 43 | if _, ok := r.byScheme[scheme]; ok { 44 | return fmt.Errorf("mount already registered for scheme: %s", scheme) 45 | } 46 | 47 | if _, ok := r.byType[reflect.TypeOf(template)]; ok { 48 | return fmt.Errorf("mount already registered for type: %T", template) 49 | } 50 | 51 | r.byScheme[scheme] = template 52 | r.byType[reflect.TypeOf(template)] = scheme 53 | return nil 54 | } 55 | 56 | // Instantiate instantiates a new Mount from a URL. 57 | // 58 | // It looks up the Mount template in the registry based on the URL scheme, 59 | // creates a copy, and calls Deserialize() on it with the supplied URL before 60 | // returning. 61 | // 62 | // It propagates any error returned by the Mount#Deserialize method. 63 | // If the scheme is not recognized, it returns ErrUnrecognizedScheme. 64 | func (r *Registry) Instantiate(u *url.URL) (Mount, error) { 65 | r.lk.RLock() 66 | defer r.lk.RUnlock() 67 | 68 | template, ok := r.byScheme[u.Scheme] 69 | if !ok { 70 | return nil, fmt.Errorf("%w: %s", ErrUnrecognizedScheme, u.Scheme) 71 | } 72 | 73 | instance := clone(template) 74 | if err := instance.Deserialize(u); err != nil { 75 | return nil, fmt.Errorf("failed to instantiate mount with url %s into type %T: %w", u.String(), template, err) 76 | } 77 | return instance, nil 78 | } 79 | 80 | // Represent returns the URL representation of a Mount, using the scheme that 81 | // was registered for that type of mount. 82 | func (r *Registry) Represent(mount Mount) (*url.URL, error) { 83 | r.lk.RLock() 84 | defer r.lk.RUnlock() 85 | 86 | // special-case the upgrader, as it's transparent. 87 | if up, ok := mount.(*Upgrader); ok { 88 | mount = up.underlying 89 | } 90 | 91 | scheme, ok := r.byType[reflect.TypeOf(mount)] 92 | if !ok { 93 | return nil, fmt.Errorf("failed to represent mount with type %T: %w", mount, ErrUnrecognizedType) 94 | } 95 | 96 | u := mount.Serialize() 97 | u.Scheme = scheme 98 | return u, nil 99 | } 100 | 101 | // clone clones m1 into m2, casting it back to a Mount. It is only able to deal 102 | // with pointer types that implement Mount. 103 | func clone(m1 Mount) (m2 Mount) { 104 | m2obj := reflect.New(reflect.TypeOf(m1).Elem()) 105 | m1val := reflect.ValueOf(m1).Elem() 106 | m2val := m2obj.Elem() 107 | for i := 0; i < m1val.NumField(); i++ { 108 | field := m2val.Field(i) 109 | if field.CanSet() { 110 | field.Set(m1val.Field(i)) 111 | } 112 | } 113 | return m2obj.Interface().(Mount) 114 | } 115 | -------------------------------------------------------------------------------- /mount/registry_test.go: -------------------------------------------------------------------------------- 1 | package mount 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "io/ioutil" 7 | "net/url" 8 | "strconv" 9 | "strings" 10 | "testing" 11 | 12 | "github.com/stretchr/testify/require" 13 | ) 14 | 15 | var _ Mount = (*MockMount)(nil) 16 | 17 | type MockMount struct { 18 | Val string 19 | URL *url.URL 20 | StatSize int64 21 | Templated string 22 | } 23 | 24 | func (m *MockMount) Serialize() *url.URL { 25 | u := &url.URL{ 26 | Scheme: "aaa", // random, will get replaced 27 | Host: m.Val, 28 | } 29 | u.Query().Set("size", strconv.FormatInt(m.StatSize, 10)) 30 | return u 31 | } 32 | 33 | func (m *MockMount) Deserialize(u *url.URL) error { 34 | vals, err := url.ParseQuery(u.RawQuery) 35 | if err != nil { 36 | return err 37 | } 38 | 39 | statSize, err := strconv.ParseInt(vals["size"][0], 10, 64) 40 | if err != nil { 41 | return err 42 | } 43 | 44 | if v, err := strconv.ParseBool(vals["timestwo"][0]); err != nil { 45 | return err 46 | } else if v { 47 | statSize *= 2 48 | } 49 | 50 | m.Val = u.Host 51 | m.URL = u 52 | m.StatSize = statSize 53 | return nil 54 | } 55 | 56 | func (m *MockMount) Close() error { 57 | panic("implement me") 58 | } 59 | 60 | func (m *MockMount) Fetch(_ context.Context) (Reader, error) { 61 | r := strings.NewReader(m.Val) 62 | return &NopCloser{Reader: r, ReaderAt: r, Seeker: r}, nil 63 | } 64 | 65 | func (m *MockMount) Info() Info { 66 | return Info{Kind: KindRemote} 67 | } 68 | 69 | func (m *MockMount) Stat(_ context.Context) (Stat, error) { 70 | return Stat{ 71 | Exists: true, 72 | Size: m.StatSize, 73 | }, nil 74 | } 75 | 76 | func TestRegistry(t *testing.T) { 77 | m1StatSize := uint64(1234) 78 | m2StatSize := uint64(5678) 79 | 80 | // create a registry 81 | r := NewRegistry() 82 | 83 | type ( 84 | MockMount1 struct{ MockMount } 85 | MockMount2 struct{ MockMount } 86 | MockMount3 struct{ MockMount } 87 | ) 88 | 89 | // create & register mock mount factory 1 90 | url1 := fmt.Sprintf("http://host1:123?size=%d×two=false", m1StatSize) 91 | u1, err := url.Parse(url1) 92 | require.NoError(t, err) 93 | require.NoError(t, r.Register("http", new(MockMount1))) 94 | // register same scheme again -> fails 95 | require.Error(t, r.Register("http", new(MockMount2))) 96 | // register same type again -> fails, different scheme 97 | require.Error(t, r.Register("http2", new(MockMount1))) 98 | 99 | // create and register mock mount factory 2 100 | url2 := fmt.Sprintf("ftp://host2:1234?size=%d×two=true", m2StatSize) 101 | u2, err := u1.Parse(url2) 102 | require.NoError(t, err) 103 | require.NoError(t, r.Register("ftp", new(MockMount3))) 104 | 105 | // instantiate mount 1 and verify state is constructed correctly 106 | m, err := r.Instantiate(u1) 107 | require.NoError(t, err) 108 | require.Equal(t, u1.Host, fetchAndReadAll(t, m)) 109 | stat, err := m.Stat(context.TODO()) 110 | require.NoError(t, err) 111 | require.EqualValues(t, m1StatSize, stat.Size) 112 | 113 | // instantiate mount 2 and verify state is constructed correctly 114 | m, err = r.Instantiate(u2) 115 | require.NoError(t, err) 116 | require.Equal(t, u2.Host, fetchAndReadAll(t, m)) 117 | stat, err = m.Stat(context.TODO()) 118 | require.NoError(t, err) 119 | require.EqualValues(t, m2StatSize*2, stat.Size) 120 | } 121 | 122 | func TestRegistryHonoursTemplate(t *testing.T) { 123 | r := NewRegistry() 124 | 125 | template := &MockMount{Templated: "give me proof"} 126 | err := r.Register("foo", template) 127 | require.NoError(t, err) 128 | 129 | u, err := url.Parse("foo://bang?size=100×two=false") 130 | require.NoError(t, err) 131 | 132 | m, err := r.Instantiate(u) 133 | require.NoError(t, err) 134 | 135 | require.Equal(t, "give me proof", m.(*MockMount).Templated) 136 | } 137 | 138 | func TestRegistryRecognizedType(t *testing.T) { 139 | type ( 140 | MockMount1 struct{ MockMount } 141 | MockMount2 struct{ MockMount } 142 | MockMount3 struct{ MockMount } 143 | ) 144 | 145 | // register all three types under different schemes 146 | r := NewRegistry() 147 | err := r.Register("mount1", new(MockMount1)) 148 | require.NoError(t, err) 149 | err = r.Register("mount2", new(MockMount2)) 150 | require.NoError(t, err) 151 | err = r.Register("mount3", new(MockMount3)) 152 | require.NoError(t, err) 153 | 154 | // now attempt to encode an instance of MockMount2 155 | u, err := r.Represent(&MockMount2{}) 156 | require.NoError(t, err) 157 | 158 | require.Equal(t, "mount2", u.Scheme) 159 | } 160 | 161 | func fetchAndReadAll(t *testing.T, m Mount) string { 162 | rd, err := m.Fetch(context.Background()) 163 | require.NoError(t, err) 164 | bz, err := ioutil.ReadAll(rd) 165 | require.NoError(t, err) 166 | return string(bz) 167 | } 168 | -------------------------------------------------------------------------------- /mount/upgrader.go: -------------------------------------------------------------------------------- 1 | package mount 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "io" 7 | "net/url" 8 | "os" 9 | "path/filepath" 10 | "sync" 11 | "sync/atomic" 12 | 13 | "github.com/filecoin-project/dagstore/throttle" 14 | logging "github.com/ipfs/go-log/v2" 15 | ) 16 | 17 | var log = logging.Logger("dagstore/upgrader") 18 | 19 | // Upgrader is a bridge to upgrade any Mount into one with full-featured 20 | // Reader capabilities, whether the original mount is of remote or local kind. 21 | // It does this by managing a local transient copy. 22 | // 23 | // If the underlying mount is fully-featured, the Upgrader has no effect, and 24 | // simply passes through to the underlying mount. 25 | type Upgrader struct { 26 | rootdir string 27 | underlying Mount 28 | throttler throttle.Throttler 29 | key string 30 | passthrough bool 31 | 32 | // paths: pathComplete is the path of transients that are 33 | // completely downloaded; pathPartial is the path where in-progress 34 | // downloads are placed. Once fully downloaded, the file is renamed to 35 | // pathComplete. 36 | pathComplete string 37 | pathPartial string 38 | 39 | lk sync.Mutex 40 | path string // guarded by lk 41 | ready bool // guarded by lk 42 | // once guards deduplicates concurrent refetch requests; the caller that 43 | // gets to run stores the result in onceErr, for other concurrent callers to 44 | // consume it. 45 | once *sync.Once // guarded by lk 46 | onceErr error // NOT guarded by lk; access coordinated by sync.Once 47 | 48 | fetches int32 // guarded by atomic 49 | } 50 | 51 | var _ Mount = (*Upgrader)(nil) 52 | 53 | // Upgrade constructs a new Upgrader for the underlying Mount. If provided, it 54 | // will reuse the file in path `initial` as the initial transient copy. Whenever 55 | // a new transient copy has to be created, it will be created under `rootdir`. 56 | func Upgrade(underlying Mount, throttler throttle.Throttler, rootdir, key string, initial string) (*Upgrader, error) { 57 | ret := &Upgrader{ 58 | underlying: underlying, 59 | key: key, 60 | rootdir: rootdir, 61 | once: new(sync.Once), 62 | throttler: throttler, 63 | pathComplete: filepath.Join(rootdir, "transient-"+key+".complete"), 64 | pathPartial: filepath.Join(rootdir, "transient-"+key+".partial"), 65 | } 66 | if ret.rootdir == "" { 67 | ret.rootdir = os.TempDir() // use the OS' default temp dir. 68 | } 69 | 70 | switch info := underlying.Info(); { 71 | case !info.AccessSequential: 72 | return nil, fmt.Errorf("underlying mount must support sequential access") 73 | case info.AccessSeek && info.AccessRandom: 74 | ret.passthrough = true 75 | return ret, nil 76 | } 77 | 78 | if initial != "" { 79 | if _, err := os.Stat(initial); err == nil { 80 | log.Debugw("initialized with existing transient that's alive", "shard", key, "path", initial) 81 | ret.path = initial 82 | ret.ready = true 83 | return ret, nil 84 | } 85 | } 86 | 87 | return ret, nil 88 | } 89 | 90 | func (u *Upgrader) Fetch(ctx context.Context) (Reader, error) { 91 | if u.passthrough { 92 | log.Debugw("fully capable mount; fetching from underlying", "shard", u.key) 93 | return u.underlying.Fetch(ctx) 94 | } 95 | 96 | // determine if the transient is still alive. 97 | // if not, delete it, get the current sync.Once and trigger a refresh. 98 | // after it's done, open the resulting transient. 99 | u.lk.Lock() 100 | if u.ready { 101 | log.Debugw("transient local copy exists; check liveness", "shard", u.key, "path", u.path) 102 | if _, err := os.Stat(u.path); err == nil { 103 | log.Debugw("transient copy alive; not refetching", "shard", u.key, "path", u.path) 104 | defer u.lk.Unlock() 105 | return os.Open(u.path) 106 | } else { 107 | u.ready = false 108 | log.Debugw("transient copy dead; removing and refetching", "shard", u.key, "path", u.path, "error", err) 109 | if err := os.Remove(u.path); err != nil { 110 | log.Warnw("refetch: failed to remove transient; garbage left behind", "shard", u.key, "dead_path", u.path, "error", err) 111 | } 112 | } 113 | // TODO add size check. 114 | } 115 | // transient appears to be dead, refetch. 116 | // get the current sync under the lock, use it to deduplicate concurrent fetches. 117 | once := u.once 118 | u.lk.Unlock() 119 | 120 | once.Do(func() { 121 | // Create a new file in the partial location. 122 | // os.Create truncates existing files. 123 | var partial *os.File 124 | partial, u.onceErr = os.Create(u.pathPartial) 125 | if u.onceErr != nil { 126 | return 127 | } 128 | defer partial.Close() 129 | 130 | // do the refetch; abort and remove/reset the partial if it fails. 131 | // perform outside the lock as this is a long-running operation. 132 | // u.onceErr is only written by the goroutine that gets to run sync.Once 133 | // and it's only read after it finishes. 134 | 135 | u.onceErr = u.refetch(ctx, partial) 136 | if u.onceErr != nil { 137 | log.Warnw("failed to refetch", "shard", u.key, "error", u.onceErr) 138 | if err := os.Remove(u.pathPartial); err != nil { 139 | log.Warnw("failed to remove partial transient", "shard", u.key, "path", u.pathPartial, "error", err) 140 | } 141 | return 142 | } 143 | 144 | // rename the partial file to a non-partial file. 145 | // set the new transient path under a lock, and recycle the sync.Once. 146 | // if the target file exists, os.Rename replaces it. 147 | if err := os.Rename(u.pathPartial, u.pathComplete); err != nil { 148 | log.Warnw("failed to rename partial transient", "shard", u.key, "from_path", u.pathPartial, "to_path", u.pathComplete, "error", err) 149 | } 150 | 151 | u.lk.Lock() 152 | u.path = u.pathComplete 153 | u.ready = true 154 | u.once = new(sync.Once) 155 | u.lk.Unlock() 156 | 157 | log.Debugw("transient path updated after refetching", "shard", u.key, "new_path", u.pathComplete) 158 | }) 159 | 160 | // There's a tiny, tiny possibility of a race here if a new refetch with 161 | // the recycled sync.Once comes in an updates onceErr before the waiters 162 | // have read it. Not worth making perfect now, but we should revisit 163 | // this recipe. Can probably use a sync.Cond with atomic counters. 164 | // TODO revisit this. 165 | err := u.onceErr 166 | if err != nil { 167 | return nil, fmt.Errorf("mount fetch failed: %w", err) 168 | } 169 | 170 | log.Debugw("refetched successfully", "shard", u.key, "path", u.pathComplete) 171 | return os.Open(u.pathComplete) 172 | } 173 | 174 | func (u *Upgrader) Info() Info { 175 | return Info{ 176 | Kind: KindLocal, 177 | AccessSequential: true, 178 | AccessSeek: true, 179 | AccessRandom: true, 180 | } 181 | } 182 | 183 | func (u *Upgrader) Stat(ctx context.Context) (Stat, error) { 184 | if u.path != "" { 185 | if stat, err := os.Stat(u.path); err == nil { 186 | ret := Stat{Exists: true, Size: stat.Size()} 187 | return ret, nil 188 | } 189 | } 190 | return u.underlying.Stat(ctx) 191 | } 192 | 193 | // TransientPath returns the local path of the transient file, if one exists. 194 | func (u *Upgrader) TransientPath() string { 195 | u.lk.Lock() 196 | defer u.lk.Unlock() 197 | 198 | return u.path 199 | } 200 | 201 | // TimesFetched returns the number of times that the underlying has 202 | // been fetched. 203 | func (u *Upgrader) TimesFetched() int { 204 | return int(atomic.LoadInt32(&u.fetches)) 205 | } 206 | 207 | // Underlying returns the underlying mount. 208 | func (u *Upgrader) Underlying() Mount { 209 | return u.underlying 210 | } 211 | 212 | func (u *Upgrader) Serialize() *url.URL { 213 | return u.underlying.Serialize() 214 | } 215 | 216 | func (u *Upgrader) Deserialize(url *url.URL) error { 217 | return u.underlying.Deserialize(url) 218 | } 219 | 220 | // TODO implement 221 | func (u *Upgrader) Close() error { 222 | log.Warnf("Upgrader.Close() not implemented yet; call will no-op") 223 | return nil 224 | } 225 | 226 | func (u *Upgrader) refetch(ctx context.Context, into *os.File) error { 227 | log.Debugw("actually refetching", "shard", u.key, "path", into.Name()) 228 | 229 | // sanity check on underlying mount. 230 | stat, err := u.underlying.Stat(ctx) 231 | if err != nil { 232 | return fmt.Errorf("underlying mount stat returned error: %w", err) 233 | } else if !stat.Exists { 234 | return fmt.Errorf("underlying mount no longer exists") 235 | } 236 | 237 | // throttle only if the file is ready; if it's not ready, we would be 238 | // throttling and then idling. 239 | t := u.throttler 240 | if !stat.Ready { 241 | log.Debugw("underlying mount is not ready; will skip throttling", "shard", u.key) 242 | t = throttle.Noop() 243 | } else { 244 | log.Debugw("underlying mount is ready; will throttle fetch and copy", "shard", u.key) 245 | } 246 | 247 | err = t.Do(ctx, func(ctx context.Context) error { 248 | // fetch from underlying and copy. 249 | from, err := u.underlying.Fetch(ctx) 250 | if err != nil { 251 | return fmt.Errorf("failed to fetch from underlying mount: %w", err) 252 | } 253 | defer from.Close() 254 | 255 | _, err = io.Copy(into, from) 256 | return err 257 | }) 258 | 259 | if err != nil { 260 | return fmt.Errorf("failed to fetch and copy underlying mount to transient file: %w", err) 261 | } 262 | 263 | return nil 264 | } 265 | 266 | // DeleteTransient deletes the transient associated with this Upgrader, if 267 | // one exists. It is the caller's responsibility to ensure the transient is 268 | // not in use. If the tracked transient is gone, this will reset the internal 269 | // state to "" (no transient) to enable recovery. 270 | func (u *Upgrader) DeleteTransient() error { 271 | u.lk.Lock() 272 | defer u.lk.Unlock() 273 | 274 | if u.path == "" { 275 | log.Debugw("transient is empty; nothing to remove", "shard", u.key) 276 | return nil // nothing to do. 277 | } 278 | 279 | // refuse to delete the transient if it's not being managed by us (i.e. in 280 | // our transients root directory). 281 | if _, err := filepath.Rel(u.rootdir, u.path); err != nil { 282 | log.Debugw("transient is not owned by us; nothing to remove", "shard", u.key) 283 | return nil 284 | } 285 | 286 | // remove the transient and clear it always, even if os.Remove 287 | // returns an error. This allows us to recover from errors like the user 288 | // deleting the transient we're currently tracking. 289 | err := os.Remove(u.path) 290 | u.path = "" 291 | u.ready = false 292 | log.Debugw("deleted existing transient", "shard", u.key, "path", u.path, "error", err) 293 | return err 294 | } 295 | -------------------------------------------------------------------------------- /mount/upgrader_test.go: -------------------------------------------------------------------------------- 1 | package mount 2 | 3 | import ( 4 | "bytes" 5 | "context" 6 | rand2 "crypto/rand" 7 | "errors" 8 | "fmt" 9 | "io" 10 | "io/ioutil" 11 | "math/rand" 12 | "net/url" 13 | "os" 14 | "sync" 15 | "sync/atomic" 16 | "testing" 17 | "time" 18 | 19 | "github.com/filecoin-project/dagstore/testdata" 20 | "github.com/filecoin-project/dagstore/throttle" 21 | "github.com/stretchr/testify/require" 22 | "golang.org/x/sync/errgroup" 23 | ) 24 | 25 | func TestUpgrade(t *testing.T) { 26 | tcs := map[string]struct { 27 | setup func(t *testing.T, key string, rootDir string) 28 | createMnt func(t *testing.T, key string, rootDir string) Mount 29 | initial string 30 | 31 | // expectations 32 | verify func(t *testing.T, u *Upgrader, key string, rootDir string) 33 | expectedContentFilePath string 34 | }{ 35 | "no transient file when underlying mount has all capabilities": { 36 | createMnt: func(t *testing.T, key string, rootDir string) Mount { 37 | return &FileMount{"../" + testdata.RootPathCarV1} 38 | }, 39 | verify: func(t *testing.T, u *Upgrader, key string, rootDir string) { 40 | fs, err := ioutil.ReadDir(rootDir) 41 | require.NoError(t, err) 42 | require.Empty(t, fs) 43 | _, err = os.Stat(u.TransientPath()) 44 | require.Error(t, err) 45 | }, 46 | expectedContentFilePath: "../" + testdata.RootPathCarV1, 47 | }, 48 | 49 | "transient file is copied from user's initial file": { 50 | initial: "../" + testdata.RootPathCarV1, 51 | 52 | createMnt: func(t *testing.T, key string, rootDir string) Mount { 53 | return &FSMount{testdata.FS, testdata.FSPathCarV2} // purposely giving a different file here. 54 | }, 55 | 56 | verify: func(t *testing.T, u *Upgrader, key string, rootDir string) { 57 | _, err := os.Stat(u.TransientPath()) 58 | require.NoError(t, err) 59 | 60 | // read the contents of the transient file. 61 | tf, err := os.Open(u.TransientPath()) 62 | require.NoError(t, err) 63 | defer tf.Close() 64 | bz, err := ioutil.ReadAll(tf) 65 | require.NoError(t, err) 66 | require.NoError(t, tf.Close()) 67 | 68 | // read the contents of the initial file -> they should match. 69 | f, err := os.Open("../" + testdata.RootPathCarV1) 70 | require.NoError(t, err) 71 | defer f.Close() 72 | bz2, err := ioutil.ReadAll(f) 73 | require.NoError(t, err) 74 | require.NoError(t, f.Close()) 75 | require.EqualValues(t, bz, bz2) 76 | }, 77 | expectedContentFilePath: "../" + testdata.RootPathCarV1, 78 | }, 79 | "delete transient": { 80 | setup: nil, 81 | createMnt: func(t *testing.T, key string, rootDir string) Mount { 82 | return &FSMount{testdata.FS, testdata.FSPathCarV2} 83 | }, 84 | verify: func(t *testing.T, u *Upgrader, key string, rootDir string) { 85 | ustat, err := u.Stat(context.TODO()) 86 | require.NoError(t, err) 87 | 88 | fstat, err := os.Stat(u.TransientPath()) 89 | require.NoError(t, err) 90 | require.EqualValues(t, fstat.Size(), ustat.Size) 91 | 92 | err = u.DeleteTransient() 93 | require.NoError(t, err) 94 | 95 | _, err = os.Stat(u.TransientPath()) 96 | require.Error(t, err) 97 | 98 | require.Empty(t, u.TransientPath()) 99 | }, 100 | expectedContentFilePath: "../" + testdata.RootPathCarV2, 101 | }, 102 | } 103 | 104 | ctx := context.Background() 105 | 106 | for name, tc := range tcs { 107 | tcc := tc 108 | t.Run(name, func(t *testing.T) { 109 | key := fmt.Sprintf("%d", rand.Uint64()) 110 | rootDir := t.TempDir() 111 | if tcc.setup != nil { 112 | tcc.setup(t, key, rootDir) 113 | } 114 | 115 | mnt := tcc.createMnt(t, key, rootDir) 116 | 117 | u, err := Upgrade(mnt, throttle.Noop(), rootDir, key, tcc.initial) 118 | require.NoError(t, err) 119 | require.NotNil(t, u) 120 | 121 | // fetch and verify contents 122 | rd, err := u.Fetch(ctx) 123 | require.NoError(t, err) 124 | require.NotNil(t, rd) 125 | 126 | bz, err := ioutil.ReadAll(rd) 127 | require.NoError(t, err) 128 | require.NotEmpty(t, bz) 129 | require.NoError(t, rd.Close()) 130 | 131 | f, err := os.Open(tcc.expectedContentFilePath) 132 | require.NoError(t, err) 133 | bz2, err := ioutil.ReadAll(f) 134 | require.NoError(t, err) 135 | require.NoError(t, f.Close()) 136 | require.EqualValues(t, bz2, bz) 137 | 138 | tcc.verify(t, u, key, rootDir) 139 | }) 140 | } 141 | } 142 | 143 | func TestUpgraderDeduplicatesRemote(t *testing.T) { 144 | ctx := context.Background() 145 | mnt := &Counting{Mount: &FSMount{testdata.FS, testdata.FSPathCarV2}} 146 | 147 | key := fmt.Sprintf("%d", rand.Uint64()) 148 | rootDir := t.TempDir() 149 | u, err := Upgrade(mnt, throttle.Noop(), rootDir, key, "") 150 | require.NoError(t, err) 151 | require.Zero(t, mnt.Count()) 152 | 153 | // now fetch in parallel 154 | cnt := 20 155 | readers := make([]Reader, cnt) 156 | grp, _ := errgroup.WithContext(context.Background()) 157 | for i := 0; i < cnt; i++ { 158 | i := i 159 | grp.Go(func() error { 160 | rd, err := u.Fetch(ctx) 161 | if err != nil { 162 | return err 163 | } 164 | readers[i] = rd 165 | return nil 166 | }) 167 | } 168 | require.NoError(t, grp.Wait()) 169 | // file should have been fetched only once 170 | require.EqualValues(t, 1, mnt.Count()) 171 | // ensure transient exists 172 | _, err = os.Stat(u.TransientPath()) 173 | require.NoError(t, err) 174 | 175 | carF, err := os.Open("../" + testdata.RootPathCarV2) 176 | require.NoError(t, err) 177 | carBytes, err := ioutil.ReadAll(carF) 178 | require.NoError(t, err) 179 | require.NoError(t, carF.Close()) 180 | 181 | grp2, _ := errgroup.WithContext(context.Background()) 182 | for _, rd := range readers { 183 | rdc := rd 184 | grp2.Go(func() error { 185 | bz, err := ioutil.ReadAll(rdc) 186 | if err != nil { 187 | return err 188 | } 189 | if err := rdc.Close(); err != nil { 190 | return err 191 | } 192 | 193 | if !bytes.Equal(carBytes, bz) { 194 | return errors.New("contents do not match") 195 | } 196 | return nil 197 | }) 198 | } 199 | require.NoError(t, grp2.Wait()) 200 | 201 | // file should have been fetched only once 202 | require.EqualValues(t, 1, mnt.Count()) 203 | 204 | // check transient still exists 205 | _, err = os.Stat(u.TransientPath()) 206 | require.NoError(t, err) 207 | 208 | // delete the transient 209 | err = os.Remove(u.TransientPath()) 210 | require.NoError(t, err) 211 | 212 | // fetch again and file should have been fetched twice 213 | rd, err := u.Fetch(ctx) 214 | require.NoError(t, err) 215 | require.EqualValues(t, 2, mnt.Count()) 216 | _, err = os.Stat(u.TransientPath()) 217 | require.NoError(t, err) 218 | 219 | require.NoError(t, rd.Close()) 220 | _, err = os.Stat(u.TransientPath()) 221 | require.NoError(t, err) 222 | } 223 | 224 | func TestUpgraderFetchAndCopyThrottle(t *testing.T) { 225 | nFixedThrottle := 3 226 | 227 | tcs := map[string]struct { 228 | ready bool 229 | expectedThrottledReads int 230 | }{ 231 | "no throttling when mount is not ready": { 232 | ready: false, 233 | expectedThrottledReads: 100, 234 | }, 235 | "throttle when mount is ready": { 236 | ready: true, 237 | expectedThrottledReads: nFixedThrottle, 238 | }, 239 | } 240 | 241 | for name, tc := range tcs { 242 | t.Run(name, func(t *testing.T) { 243 | thrt := throttle.Fixed(nFixedThrottle) // same throttle for all 244 | ctx := context.Background() 245 | 246 | upgraders := make([]*Upgrader, 100) 247 | 248 | underlyings := make([]*blockingReaderMount, 100) 249 | for i := range upgraders { 250 | underlyings[i] = &blockingReaderMount{isReady: tc.ready, br: &blockingReader{r: io.LimitReader(rand2.Reader, 1)}} 251 | u, err := Upgrade(underlyings[i], thrt, t.TempDir(), "foo", "") 252 | require.NoError(t, err) 253 | upgraders[i] = u 254 | } 255 | 256 | // take all locks. 257 | for _, uu := range underlyings { 258 | uu.br.lk.Lock() 259 | } 260 | 261 | errgrp, _ := errgroup.WithContext(ctx) 262 | for _, u := range upgraders { 263 | u := u 264 | errgrp.Go(func() error { 265 | _, err := u.Fetch(ctx) 266 | return err 267 | }) 268 | } 269 | 270 | time.Sleep(500 * time.Millisecond) 271 | 272 | // calls to read across all readers are made without throttling. 273 | var total int32 274 | for _, uu := range underlyings { 275 | total += atomic.LoadInt32(&uu.br.reads) 276 | } 277 | 278 | require.EqualValues(t, tc.expectedThrottledReads, total) 279 | 280 | // release all locks. 281 | for _, uu := range underlyings { 282 | uu.br.lk.Unlock() 283 | } 284 | 285 | require.NoError(t, errgrp.Wait()) 286 | 287 | // we expect 200 calls to read across all readers. 288 | // 2 per reader: fetching the byte, and the EOF. 289 | total = 0 290 | for _, uu := range underlyings { 291 | total += atomic.LoadInt32(&uu.br.reads) 292 | } 293 | 294 | require.EqualValues(t, 200, total) // all accessed 295 | }) 296 | } 297 | } 298 | 299 | type blockingReader struct { 300 | r io.Reader 301 | lk sync.Mutex 302 | reads int32 303 | } 304 | 305 | var _ Reader = (*blockingReader)(nil) 306 | 307 | func (br *blockingReader) Close() error { 308 | return nil 309 | } 310 | 311 | func (br *blockingReader) ReadAt(p []byte, off int64) (n int, err error) { 312 | panic("implement me") 313 | } 314 | 315 | func (br *blockingReader) Seek(offset int64, whence int) (int64, error) { 316 | panic("implement me") 317 | } 318 | 319 | func (br *blockingReader) Read(b []byte) (n int, err error) { 320 | atomic.AddInt32(&br.reads, 1) 321 | br.lk.Lock() 322 | defer br.lk.Unlock() 323 | n, err = br.r.Read(b) 324 | return n, err 325 | } 326 | 327 | type blockingReaderMount struct { 328 | isReady bool 329 | br *blockingReader 330 | } 331 | 332 | var _ Mount = (*blockingReaderMount)(nil) 333 | 334 | func (b *blockingReaderMount) Close() error { 335 | return nil 336 | } 337 | 338 | func (b *blockingReaderMount) Fetch(ctx context.Context) (Reader, error) { 339 | return b.br, nil 340 | } 341 | 342 | func (b *blockingReaderMount) Info() Info { 343 | return Info{ 344 | Kind: KindRemote, 345 | AccessSequential: true, 346 | } 347 | } 348 | 349 | func (b *blockingReaderMount) Stat(ctx context.Context) (Stat, error) { 350 | return Stat{ 351 | Exists: true, 352 | Size: 1024, 353 | Ready: b.isReady, 354 | }, nil 355 | } 356 | 357 | func (b *blockingReaderMount) Serialize() *url.URL { 358 | panic("implement me") 359 | } 360 | 361 | func (b *blockingReaderMount) Deserialize(url *url.URL) error { 362 | panic("implement me") 363 | } 364 | -------------------------------------------------------------------------------- /shard.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | import ( 4 | "context" 5 | "sync" 6 | 7 | "github.com/filecoin-project/dagstore/mount" 8 | "github.com/filecoin-project/dagstore/shard" 9 | ) 10 | 11 | // waiter encapsulates a context passed by the user, and the channel they want 12 | // the result returned to. 13 | type waiter struct { 14 | ctx context.Context // governs the op if it's external 15 | outCh chan<- ShardResult // to send back the result 16 | notifyDead func() // called when the context expired and we weren't able to deliver the result 17 | } 18 | 19 | func (w waiter) deliver(res *ShardResult) { 20 | if w.outCh == nil { 21 | return 22 | } 23 | select { 24 | case w.outCh <- *res: 25 | case <-w.ctx.Done(): 26 | if w.notifyDead != nil { 27 | w.notifyDead() 28 | } 29 | } 30 | } 31 | 32 | // Shard encapsulates the state of a shard within the DAG store. 33 | type Shard struct { 34 | lk sync.RWMutex 35 | 36 | // Immutable fields. 37 | // Safe to read outside the event loop without a lock. 38 | d *DAGStore // backreference 39 | key shard.Key // persisted in PersistedShard.Key 40 | mount *mount.Upgrader // persisted in PersistedShard.URL (underlying) 41 | lazy bool // persisted in PersistedShard.Lazy; whether this shard has lazy indexing 42 | 43 | // Mutable fields. 44 | // Cannot read/write outside event loop. 45 | state ShardState // persisted in PersistedShard.State 46 | err error // persisted in PersistedShard.Error; populated if shard state is errored. 47 | 48 | recoverOnNextAcquire bool // a shard marked in error state during initialization can be recovered on its first acquire. 49 | 50 | // Waiters. 51 | wRegister *waiter // waiter for registration result. 52 | wRecover *waiter // waiter for recovering an errored shard. 53 | wAcquire []*waiter // waiters for acquiring the shard. 54 | wDestroy *waiter // waiter for shard destruction. 55 | 56 | refs uint32 // number of DAG accessors currently open 57 | } 58 | -------------------------------------------------------------------------------- /shard/key.go: -------------------------------------------------------------------------------- 1 | package shard 2 | 3 | import ( 4 | "encoding/json" 5 | 6 | "github.com/ipfs/go-cid" 7 | "github.com/mr-tron/base58" 8 | ) 9 | 10 | // Key represents a shard key. It can be instantiated from a string, a byte 11 | // slice, or a CID. 12 | type Key struct { 13 | // str stores a string or an arbitrary byte slice in base58 form. We cannot 14 | // store the raw byte slice because this struct needs to be comparable. 15 | str string 16 | } 17 | 18 | // KeyFromString returns a key representing an arbitrary string. 19 | func KeyFromString(str string) Key { 20 | return Key{str: str} 21 | } 22 | 23 | // KeyFromBytes returns a key from a byte slice, encoding it in b58 first. 24 | func KeyFromBytes(b []byte) Key { 25 | return Key{str: base58.Encode(b)} 26 | } 27 | 28 | // KeyFromCID returns a key representing a CID. 29 | func KeyFromCID(cid cid.Cid) Key { 30 | return Key{str: cid.String()} 31 | } 32 | 33 | // String returns the string representation for this key. 34 | func (k Key) String() string { 35 | return k.str 36 | } 37 | 38 | // 39 | // We need a custom JSON marshaller and unmarshaller because str is a 40 | // private field 41 | // 42 | func (k Key) MarshalJSON() ([]byte, error) { 43 | return json.Marshal(k.str) 44 | } 45 | 46 | func (k *Key) UnmarshalJSON(bz []byte) error { 47 | return json.Unmarshal(bz, &k.str) 48 | } 49 | -------------------------------------------------------------------------------- /shard/key_test.go: -------------------------------------------------------------------------------- 1 | package shard 2 | 3 | import ( 4 | "encoding/json" 5 | "testing" 6 | 7 | "github.com/stretchr/testify/require" 8 | ) 9 | 10 | func TestKeyMarshalJSON(t *testing.T) { 11 | k := Key{"abc"} 12 | bz, err := json.Marshal(k) 13 | require.NoError(t, err) 14 | 15 | var k2 Key 16 | err = json.Unmarshal(bz, &k2) 17 | require.NoError(t, err) 18 | 19 | require.Equal(t, k.str, k2.str) 20 | } 21 | -------------------------------------------------------------------------------- /shard_persist.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | import ( 4 | "context" 5 | "encoding/json" 6 | "errors" 7 | "fmt" 8 | "net/url" 9 | 10 | "github.com/filecoin-project/dagstore/mount" 11 | "github.com/filecoin-project/dagstore/shard" 12 | ds "github.com/ipfs/go-datastore" 13 | ) 14 | 15 | // PersistedShard is the persistent representation of the Shard. 16 | type PersistedShard struct { 17 | Key string `json:"k"` 18 | URL string `json:"u"` 19 | TransientPath string `json:"t"` 20 | State ShardState `json:"s"` 21 | Lazy bool `json:"l"` 22 | Error string `json:"e"` 23 | } 24 | 25 | // MarshalJSON returns a serialized representation of the state. It must be 26 | // called with a shard lock (read, at least), such as from inside the event 27 | // loop, as it accesses mutable state. 28 | func (s *Shard) MarshalJSON() ([]byte, error) { 29 | u, err := s.d.mounts.Represent(s.mount) 30 | if err != nil { 31 | return nil, fmt.Errorf("failed to encode mount: %w", err) 32 | } 33 | ps := PersistedShard{ 34 | Key: s.key.String(), 35 | URL: u.String(), 36 | State: s.state, 37 | Lazy: s.lazy, 38 | TransientPath: s.mount.TransientPath(), 39 | } 40 | if s.err != nil { 41 | ps.Error = s.err.Error() 42 | } 43 | 44 | return json.Marshal(ps) 45 | // TODO maybe switch to CBOR, as it's probably faster. 46 | // var b bytes.Buffer 47 | // if err := ps.MarshalCBOR(&b); err != nil { 48 | // return nil, err 49 | // } 50 | // return b.Bytes(), nil 51 | } 52 | 53 | func (s *Shard) UnmarshalJSON(b []byte) error { 54 | var ps PersistedShard // TODO try to avoid this alloc by marshalling/unmarshalling directly. 55 | if err := json.Unmarshal(b, &ps); err != nil { 56 | return err 57 | } 58 | 59 | // restore basics. 60 | s.key = shard.KeyFromString(ps.Key) 61 | s.state = ps.State 62 | s.lazy = ps.Lazy 63 | if ps.Error != "" { 64 | s.err = errors.New(ps.Error) 65 | } 66 | 67 | // restore mount. 68 | u, err := url.Parse(ps.URL) 69 | if err != nil { 70 | return fmt.Errorf("failed to parse mount URL: %w", err) 71 | } 72 | mnt, err := s.d.mounts.Instantiate(u) 73 | if err != nil { 74 | return fmt.Errorf("failed to instantiate mount from URL: %w", err) 75 | } 76 | s.mount, err = mount.Upgrade(mnt, s.d.throttleReaadyFetch, s.d.config.TransientsDir, s.key.String(), ps.TransientPath) 77 | if err != nil { 78 | return fmt.Errorf("failed to apply mount upgrader: %w", err) 79 | } 80 | 81 | return nil 82 | } 83 | 84 | // persist persists the shard's state into the supplied Datastore. It calls 85 | // MarshalJSON, which requires holding a shard lock to be safe. 86 | func (s *Shard) persist(ctx context.Context, store ds.Datastore) error { 87 | ps, err := s.MarshalJSON() 88 | if err != nil { 89 | return fmt.Errorf("failed to serialize shard state: %w", err) 90 | } 91 | // assuming that the datastore is namespaced if need be. 92 | k := ds.NewKey(s.key.String()) 93 | if err := store.Put(ctx, k, ps); err != nil { 94 | return fmt.Errorf("failed to put shard state: %w", err) 95 | } 96 | if err := store.Sync(ctx, ds.Key{}); err != nil { 97 | return fmt.Errorf("failed to sync shard state to store: %w", err) 98 | } 99 | return nil 100 | } 101 | -------------------------------------------------------------------------------- /shard_state.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | type ShardState byte 4 | 5 | const ( 6 | // ShardStateNew indicates that a shard has just been registered and is 7 | // about to be processed for activation. 8 | ShardStateNew ShardState = iota 9 | 10 | // ShardStateInitializing indicates that the shard is being initialized 11 | // by being fetched from the mount and being indexed. 12 | ShardStateInitializing 13 | 14 | // ShardStateAvailable indicates that the shard has been initialized and is 15 | // active for serving queries. There are no active shard readers. 16 | ShardStateAvailable 17 | 18 | // ShardStateServing indicates the shard has active readers and thus is 19 | // currently actively serving requests. 20 | ShardStateServing 21 | 22 | // ShardStateRecovering indicates that the shard is recovering from an 23 | // errored state. Such recoveries are always initiated by the user through 24 | // DAGStore.RecoverShard(). 25 | ShardStateRecovering ShardState = 0x80 26 | 27 | // ShardStateErrored indicates that an unexpected error was encountered 28 | // during a shard operation, and therefore the shard needs to be recovered. 29 | ShardStateErrored ShardState = 0xf0 30 | 31 | // ShardStateUnknown indicates that it's not possible to determine the state 32 | // of the shard. This state is currently unused, but it's reserved. 33 | ShardStateUnknown ShardState = 0xff 34 | ) 35 | 36 | func (ss ShardState) String() string { 37 | strs := [...]string{ 38 | ShardStateNew: "ShardStateNew", 39 | ShardStateInitializing: "ShardStateInitializing", 40 | ShardStateAvailable: "ShardStateAvailable", 41 | ShardStateServing: "ShardStateServing", 42 | ShardStateRecovering: "ShardStateRecovering", 43 | ShardStateErrored: "ShardStateErrored", 44 | ShardStateUnknown: "ShardStateUnknown", 45 | } 46 | if ss < 0 || int(ss) >= len(strs) { 47 | // safety comes first. 48 | return "__undefined__" 49 | } 50 | return strs[ss] 51 | } 52 | -------------------------------------------------------------------------------- /shard_state_test.go: -------------------------------------------------------------------------------- 1 | package dagstore 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/require" 7 | ) 8 | 9 | func TestShardStates(t *testing.T) { 10 | ss := ShardStateRecovering 11 | require.Equal(t, "ShardStateRecovering", ss.String()) 12 | 13 | ss = ShardStateInitializing 14 | require.Equal(t, "ShardStateInitializing", ss.String()) 15 | 16 | ss = ShardState(201) 17 | require.Equal(t, "", ss.String()) 18 | } 19 | -------------------------------------------------------------------------------- /testdata/files/junk.dat: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/filecoin-project/dagstore/67a2ac2b7bdd71ee69569745dfc5d3106ec4c68b/testdata/files/junk.dat -------------------------------------------------------------------------------- /testdata/files/sample-v1.car: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/filecoin-project/dagstore/67a2ac2b7bdd71ee69569745dfc5d3106ec4c68b/testdata/files/sample-v1.car -------------------------------------------------------------------------------- /testdata/files/sample-wrapped-v2.car: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/filecoin-project/dagstore/67a2ac2b7bdd71ee69569745dfc5d3106ec4c68b/testdata/files/sample-wrapped-v2.car -------------------------------------------------------------------------------- /testdata/testdata.go: -------------------------------------------------------------------------------- 1 | package testdata 2 | 3 | import ( 4 | "bytes" 5 | "context" 6 | "embed" 7 | "fmt" 8 | "io" 9 | "math/rand" 10 | "os" 11 | 12 | "github.com/ipfs/go-blockservice" 13 | "github.com/ipfs/go-cid" 14 | "github.com/ipfs/go-cidutil" 15 | ds "github.com/ipfs/go-datastore" 16 | dssync "github.com/ipfs/go-datastore/sync" 17 | bstore "github.com/ipfs/go-ipfs-blockstore" 18 | chunk "github.com/ipfs/go-ipfs-chunker" 19 | offline "github.com/ipfs/go-ipfs-exchange-offline" 20 | files "github.com/ipfs/go-ipfs-files" 21 | ipldformat "github.com/ipfs/go-ipld-format" 22 | "github.com/ipfs/go-merkledag" 23 | "github.com/ipfs/go-unixfs/importer/balanced" 24 | ihelper "github.com/ipfs/go-unixfs/importer/helpers" 25 | "github.com/ipld/go-car/v2" 26 | "github.com/ipld/go-car/v2/blockstore" 27 | "github.com/multiformats/go-multihash" 28 | ) 29 | 30 | const ( 31 | FSPathCarV1 = "files/sample-v1.car" 32 | FSPathCarV2 = "files/sample-wrapped-v2.car" 33 | FSPathJunk = "files/junk.dat" 34 | 35 | RootPathCarV1 = "testdata/files/sample-v1.car" 36 | RootPathCarV2 = "testdata/files/sample-wrapped-v2.car" 37 | RootPathJunk = "testdata/files/funk.dat" 38 | 39 | defaultHashFunction = uint64(multihash.BLAKE2B_MIN + 31) 40 | unixfsChunkSize = uint64(1 << 10) 41 | unixfsLinksPerLevel = 1024 42 | ) 43 | 44 | var ( 45 | //go:embed files/* 46 | FS embed.FS 47 | 48 | CarV1 []byte 49 | CarV2 []byte 50 | Junk []byte 51 | 52 | // RootCID is the root CID of the carv2 for testing. 53 | RootCID cid.Cid 54 | ) 55 | 56 | func init() { 57 | var err error 58 | CarV1, err = FS.ReadFile(FSPathCarV1) 59 | if err != nil { 60 | panic(err) 61 | } 62 | 63 | CarV2, err = FS.ReadFile(FSPathCarV2) 64 | if err != nil { 65 | panic(err) 66 | } 67 | 68 | Junk, err = FS.ReadFile(FSPathJunk) 69 | if err != nil { 70 | panic(err) 71 | } 72 | 73 | reader, err := car.NewReader(bytes.NewReader(CarV2)) 74 | if err != nil { 75 | panic(fmt.Errorf("failed to parse carv2: %w", err)) 76 | } 77 | defer reader.Close() 78 | 79 | roots, err := reader.Roots() 80 | if err != nil { 81 | panic(fmt.Errorf("failed to obtain carv2 roots: %w", err)) 82 | } 83 | if len(roots) == 0 { 84 | panic("carv2 has no roots") 85 | } 86 | RootCID = roots[0] 87 | } 88 | 89 | func CreateRandomFile(dir string, rseed, size int) (string, error) { 90 | source := io.LimitReader(rand.New(rand.NewSource(int64(rseed))), int64(size)) 91 | 92 | file, err := os.CreateTemp(dir, "sourcefile.dat") 93 | if err != nil { 94 | return "", err 95 | } 96 | 97 | _, err = io.Copy(file, source) 98 | if err != nil { 99 | return "", err 100 | } 101 | 102 | // 103 | _, err = file.Seek(0, io.SeekStart) 104 | if err != nil { 105 | return "", err 106 | } 107 | 108 | return file.Name(), nil 109 | } 110 | 111 | // CreateDenseCARv2 generates a "dense" UnixFS CARv2 from the supplied ordinary file. 112 | // A dense UnixFS CARv2 is one storing leaf data. Contrast to CreateRefCARv2. 113 | func CreateDenseCARv2(dir, src string) (cid.Cid, string, error) { 114 | bs := bstore.NewBlockstore(dssync.MutexWrap(ds.NewMapDatastore())) 115 | dagSvc := merkledag.NewDAGService(blockservice.New(bs, offline.Exchange(bs))) 116 | 117 | root, err := WriteUnixfsDAGTo(src, dagSvc) 118 | if err != nil { 119 | return cid.Undef, "", err 120 | } 121 | 122 | // Create a UnixFS DAG again AND generate a CARv2 file using a CARv2 123 | // read-write blockstore now that we have the root. 124 | out, err := os.CreateTemp(dir, "rand") 125 | if err != nil { 126 | return cid.Undef, "", err 127 | } 128 | err = out.Close() 129 | if err != nil { 130 | return cid.Undef, "", err 131 | } 132 | 133 | rw, err := blockstore.OpenReadWrite(out.Name(), []cid.Cid{root}, blockstore.UseWholeCIDs(true)) 134 | if err != nil { 135 | return cid.Undef, "", err 136 | } 137 | 138 | dagSvc = merkledag.NewDAGService(blockservice.New(rw, offline.Exchange(rw))) 139 | 140 | root2, err := WriteUnixfsDAGTo(src, dagSvc) 141 | if err != nil { 142 | return cid.Undef, "", err 143 | } 144 | 145 | err = rw.Finalize() 146 | if err != nil { 147 | return cid.Undef, "", err 148 | } 149 | 150 | if root != root2 { 151 | return cid.Undef, "", fmt.Errorf("DAG root cid mismatch") 152 | } 153 | 154 | return root, out.Name(), nil 155 | } 156 | 157 | func WriteUnixfsDAGTo(path string, into ipldformat.DAGService) (cid.Cid, error) { 158 | file, err := os.Open(path) 159 | if err != nil { 160 | return cid.Undef, err 161 | } 162 | defer file.Close() 163 | 164 | stat, err := file.Stat() 165 | if err != nil { 166 | return cid.Undef, err 167 | } 168 | 169 | // get a IPLD reader path file 170 | // required to write the Unixfs DAG blocks to a filestore 171 | rpf, err := files.NewReaderPathFile(file.Name(), file, stat) 172 | if err != nil { 173 | return cid.Undef, err 174 | } 175 | 176 | // generate the dag and get the root 177 | // import to UnixFS 178 | prefix, err := merkledag.PrefixForCidVersion(1) 179 | if err != nil { 180 | return cid.Undef, err 181 | } 182 | 183 | prefix.MhType = defaultHashFunction 184 | 185 | bufferedDS := ipldformat.NewBufferedDAG(context.Background(), into) 186 | params := ihelper.DagBuilderParams{ 187 | Maxlinks: unixfsLinksPerLevel, 188 | RawLeaves: true, 189 | // NOTE: InlineBuilder not recommended, we are using this to test identity CIDs 190 | CidBuilder: cidutil.InlineBuilder{ 191 | Builder: prefix, 192 | Limit: 126, 193 | }, 194 | Dagserv: bufferedDS, 195 | NoCopy: true, 196 | } 197 | 198 | db, err := params.New(chunk.NewSizeSplitter(rpf, int64(unixfsChunkSize))) 199 | if err != nil { 200 | return cid.Undef, err 201 | } 202 | 203 | nd, err := balanced.Layout(db) 204 | if err != nil { 205 | return cid.Undef, err 206 | } 207 | 208 | err = bufferedDS.Commit() 209 | if err != nil { 210 | return cid.Undef, err 211 | } 212 | 213 | err = rpf.Close() 214 | if err != nil { 215 | return cid.Undef, err 216 | } 217 | 218 | return nd.Cid(), nil 219 | } 220 | -------------------------------------------------------------------------------- /throttle/doc.go: -------------------------------------------------------------------------------- 1 | // Package throttle includes throttlers for composing in various contexts, such 2 | // as inside Mounts for costly operations, and within the DAG store itself. 3 | package throttle 4 | -------------------------------------------------------------------------------- /throttle/throttler.go: -------------------------------------------------------------------------------- 1 | package throttle 2 | 3 | import "context" 4 | 5 | // Throttler is a component to perform throttling of concurrent requests. 6 | type Throttler interface { 7 | // Do performs the supplied action under the guard of the throttler. 8 | // 9 | // The supplied context is obeyed when parking to claim a throttler spot, and is 10 | // passed to the action. Errors from the action are propagated to the caller, 11 | // as are context deadline errors. 12 | // 13 | // Do blocks until the action has executed. 14 | Do(context.Context, func(ctx context.Context) error) error 15 | } 16 | 17 | type throttler struct { 18 | ch chan struct{} 19 | } 20 | 21 | // Fixed creates a new throttler that allows the specified fixed concurrency 22 | // at most. 23 | func Fixed(maxConcurrency int) Throttler { 24 | ch := make(chan struct{}, maxConcurrency) 25 | for i := 0; i < maxConcurrency; i++ { 26 | ch <- struct{}{} 27 | } 28 | return &throttler{ch: ch} 29 | } 30 | 31 | func (t *throttler) Do(ctx context.Context, fn func(ctx context.Context) error) error { 32 | select { 33 | case <-t.ch: 34 | case <-ctx.Done(): 35 | return ctx.Err() 36 | } 37 | defer func() { t.ch <- struct{}{} }() 38 | return fn(ctx) 39 | } 40 | 41 | // Noop returns a noop throttler. 42 | func Noop() Throttler { 43 | return noopThrottler{} 44 | } 45 | 46 | type noopThrottler struct{} 47 | 48 | func (noopThrottler) Do(ctx context.Context, fn func(ctx context.Context) error) error { 49 | return fn(ctx) 50 | } 51 | -------------------------------------------------------------------------------- /throttle/throttler_test.go: -------------------------------------------------------------------------------- 1 | package throttle 2 | 3 | import ( 4 | "context" 5 | "sync/atomic" 6 | "testing" 7 | "time" 8 | 9 | "github.com/stretchr/testify/require" 10 | "golang.org/x/sync/errgroup" 11 | ) 12 | 13 | func TestThrottler(t *testing.T) { 14 | tt := Fixed(5) 15 | 16 | var cnt int32 17 | ch := make(chan struct{}, 16) 18 | fn := func(ctx context.Context) error { 19 | <-ch 20 | atomic.AddInt32(&cnt, 1) 21 | return nil 22 | } 23 | 24 | // spawn 10 processes; all of them will block consuming from ch. 25 | grp, _ := errgroup.WithContext(context.Background()) 26 | for i := 0; i < 10; i++ { 27 | grp.Go(func() error { 28 | return tt.Do(context.Background(), fn) 29 | }) 30 | } 31 | 32 | time.Sleep(100 * time.Millisecond) 33 | 34 | // counter is still 0. 35 | require.Zero(t, atomic.LoadInt32(&cnt)) 36 | 37 | // allow 5 to proceed and unblock. 38 | for i := 0; i < 5; i++ { 39 | ch <- struct{}{} 40 | } 41 | 42 | time.Sleep(100 * time.Millisecond) 43 | 44 | // counter is 5 but not 10. 45 | require.EqualValues(t, 5, atomic.LoadInt32(&cnt)) 46 | 47 | // spawn another 10. 48 | for i := 0; i < 10; i++ { 49 | grp.Go(func() error { 50 | return tt.Do(context.Background(), fn) 51 | }) 52 | } 53 | 54 | // allow 10 to proceed and unblock. 55 | for i := 0; i < 10; i++ { 56 | ch <- struct{}{} 57 | } 58 | 59 | time.Sleep(100 * time.Millisecond) 60 | 61 | // counter is 15 but not 20. 62 | require.EqualValues(t, 15, atomic.LoadInt32(&cnt)) 63 | 64 | // test with a cancelled context. 65 | ctx, cancel := context.WithCancel(context.Background()) 66 | errCh := make(chan error) 67 | for i := 0; i < 10; i++ { 68 | go func() { 69 | errCh <- tt.Do(ctx, fn) 70 | }() 71 | } 72 | time.Sleep(100 * time.Millisecond) 73 | cancel() 74 | 75 | for i := 0; i < 10; i++ { 76 | require.ErrorIs(t, <-errCh, context.Canceled) 77 | } 78 | } 79 | --------------------------------------------------------------------------------