ipfs-cluster/pintracker/optracker/operationtracker.go

402 lines
11 KiB
Go
Raw Normal View History

Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
// Package optracker implements functionality to track the status of pin and
// operations as needed by implementations of the pintracker component.
// It particularly allows to obtain status information for a given Cid,
// to skip re-tracking already ongoing operations, or to cancel ongoing
// operations when opposing ones arrive.
package optracker
import (
"context"
"fmt"
"strings"
"sync"
"sync/atomic"
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
"time"
"github.com/ipfs-cluster/ipfs-cluster/api"
"github.com/ipfs-cluster/ipfs-cluster/observations"
logging "github.com/ipfs/go-log/v2"
Dependency upgrades (#1755) * Update go-libp2p to v0.22.0 * Testing with go1.19 * build(deps): bump github.com/multiformats/go-multicodec Bumps [github.com/multiformats/go-multicodec](https://github.com/multiformats/go-multicodec) from 0.5.0 to 0.6.0. - [Release notes](https://github.com/multiformats/go-multicodec/releases) - [Commits](https://github.com/multiformats/go-multicodec/compare/v0.5.0...v0.6.0) --- updated-dependencies: - dependency-name: github.com/multiformats/go-multicodec dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> * build(deps): bump github.com/ipld/go-car from 0.4.0 to 0.5.0 Bumps [github.com/ipld/go-car](https://github.com/ipld/go-car) from 0.4.0 to 0.5.0. - [Release notes](https://github.com/ipld/go-car/releases) - [Commits](https://github.com/ipld/go-car/compare/v0.4.0...v0.5.0) --- updated-dependencies: - dependency-name: github.com/ipld/go-car dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> * build(deps): bump github.com/prometheus/client_golang Bumps [github.com/prometheus/client_golang](https://github.com/prometheus/client_golang) from 1.12.2 to 1.13.0. - [Release notes](https://github.com/prometheus/client_golang/releases) - [Changelog](https://github.com/prometheus/client_golang/blob/main/CHANGELOG.md) - [Commits](https://github.com/prometheus/client_golang/compare/v1.12.2...v1.13.0) --- updated-dependencies: - dependency-name: github.com/prometheus/client_golang dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> * build(deps): bump github.com/hashicorp/go-hclog from 1.2.1 to 1.3.0 Bumps [github.com/hashicorp/go-hclog](https://github.com/hashicorp/go-hclog) from 1.2.1 to 1.3.0. - [Release notes](https://github.com/hashicorp/go-hclog/releases) - [Commits](https://github.com/hashicorp/go-hclog/compare/v1.2.1...v1.3.0) --- updated-dependencies: - dependency-name: github.com/hashicorp/go-hclog dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> * build(deps): bump github.com/ipfs/go-ds-crdt from 0.3.6 to 0.3.7 Bumps [github.com/ipfs/go-ds-crdt](https://github.com/ipfs/go-ds-crdt) from 0.3.6 to 0.3.7. - [Release notes](https://github.com/ipfs/go-ds-crdt/releases) - [Commits](https://github.com/ipfs/go-ds-crdt/compare/v0.3.6...v0.3.7) --- updated-dependencies: - dependency-name: github.com/ipfs/go-ds-crdt dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] <support@github.com> * build(deps): bump github.com/urfave/cli/v2 from 2.10.2 to 2.14.1 Bumps [github.com/urfave/cli/v2](https://github.com/urfave/cli) from 2.10.2 to 2.14.1. - [Release notes](https://github.com/urfave/cli/releases) - [Changelog](https://github.com/urfave/cli/blob/main/docs/CHANGELOG.md) - [Commits](https://github.com/urfave/cli/compare/v2.10.2...v2.14.1) --- updated-dependencies: - dependency-name: github.com/urfave/cli/v2 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> * build(deps): bump github.com/libp2p/go-libp2p-http from 0.3.0 to 0.4.0 Bumps [github.com/libp2p/go-libp2p-http](https://github.com/libp2p/go-libp2p-http) from 0.3.0 to 0.4.0. - [Release notes](https://github.com/libp2p/go-libp2p-http/releases) - [Commits](https://github.com/libp2p/go-libp2p-http/compare/v0.3.0...v0.4.0) --- updated-dependencies: - dependency-name: github.com/libp2p/go-libp2p-http dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> * build(deps): bump github.com/libp2p/go-libp2p-gorpc from 0.4.0 to 0.5.0 Bumps [github.com/libp2p/go-libp2p-gorpc](https://github.com/libp2p/go-libp2p-gorpc) from 0.4.0 to 0.5.0. - [Release notes](https://github.com/libp2p/go-libp2p-gorpc/releases) - [Commits](https://github.com/libp2p/go-libp2p-gorpc/compare/v0.4.0...v0.5.0) --- updated-dependencies: - dependency-name: github.com/libp2p/go-libp2p-gorpc dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> * build(deps): bump contrib.go.opencensus.io/exporter/prometheus Bumps [contrib.go.opencensus.io/exporter/prometheus](https://github.com/census-ecosystem/opencensus-go-exporter-prometheus) from 0.4.1 to 0.4.2. - [Release notes](https://github.com/census-ecosystem/opencensus-go-exporter-prometheus/releases) - [Commits](https://github.com/census-ecosystem/opencensus-go-exporter-prometheus/compare/v0.4.1...v0.4.2) --- updated-dependencies: - dependency-name: contrib.go.opencensus.io/exporter/prometheus dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] <support@github.com> * build(deps): bump github.com/libp2p/go-libp2p-raft from 0.1.8 to 0.2.0 Bumps [github.com/libp2p/go-libp2p-raft](https://github.com/libp2p/go-libp2p-raft) from 0.1.8 to 0.2.0. - [Release notes](https://github.com/libp2p/go-libp2p-raft/releases) - [Commits](https://github.com/libp2p/go-libp2p-raft/compare/v0.1.8...v0.2.0) --- updated-dependencies: - dependency-name: github.com/libp2p/go-libp2p-raft dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> * build(deps): bump github.com/urfave/cli from 1.22.9 to 1.22.10 Bumps [github.com/urfave/cli](https://github.com/urfave/cli) from 1.22.9 to 1.22.10. - [Release notes](https://github.com/urfave/cli/releases) - [Changelog](https://github.com/urfave/cli/blob/main/docs/CHANGELOG.md) - [Commits](https://github.com/urfave/cli/compare/v1.22.9...v1.22.10) --- updated-dependencies: - dependency-name: github.com/urfave/cli dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] <support@github.com> * Fix checker/linter/staticcheck warnings Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
2022-09-06 14:57:17 +00:00
peer "github.com/libp2p/go-libp2p/core/peer"
"go.opencensus.io/stats"
"go.opencensus.io/trace"
)
var logger = logging.Logger("optracker")
// OperationTracker tracks and manages all inflight Operations.
type OperationTracker struct {
// struct alignment. This fields must be upfront!
pinningCount int64
pinErrorCount int64
pinQueuedCount int64
ctx context.Context // parent context for all ops
pid peer.ID
peerName string
mu sync.RWMutex
operations map[api.Cid]*Operation
}
func (opt *OperationTracker) String() string {
var b strings.Builder
fmt.Fprintf(&b, "pid: %v\n", opt.pid)
fmt.Fprintf(&b, "name: %s\n", opt.peerName)
fmt.Fprint(&b, "operations:\n")
opt.mu.RLock()
defer opt.mu.RUnlock()
for _, op := range opt.operations {
opstr := op.String()
opstrs := strings.Split(opstr, "\n")
for _, s := range opstrs {
fmt.Fprintf(&b, "\t%s\n", s)
}
}
return b.String()
}
// NewOperationTracker creates a new OperationTracker.
func NewOperationTracker(ctx context.Context, pid peer.ID, peerName string) *OperationTracker {
initializeMetrics(ctx)
return &OperationTracker{
ctx: ctx,
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
pid: pid,
peerName: peerName,
operations: make(map[api.Cid]*Operation),
}
}
// TrackNewOperation will create, track and return a new operation unless
// one already exists to do the same thing, in which case nil is returned.
//
// If an operation exists it is of different type, it is
// canceled and the new one replaces it in the tracker.
func (opt *OperationTracker) TrackNewOperation(ctx context.Context, pin api.Pin, typ OperationType, ph Phase) *Operation {
_, span := trace.StartSpan(ctx, "optracker/TrackNewOperation")
defer span.End()
opt.mu.Lock()
defer opt.mu.Unlock()
op, ok := opt.operations[pin.Cid]
if ok { // operation exists for the CID
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
if op.Type() == typ && op.Phase() != PhaseError && op.Phase() != PhaseDone {
// an ongoing operation of the same
// type. i.e. pinning, or queued. Update the pin
// object though, as it may have different options.
op.pin = pin
return nil
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
}
// i.e. operations in error phase
// i.e. pin operations that need to be canceled for unpinning
op.tracker.recordMetric(op, -1)
op.Cancel() // cancel ongoing operation and replace it
}
// IMPORTANT: the operations must have the OperationTracker context,
// as otherwise their context would be canceled after being added.
op2 := newOperation(opt.ctx, pin, typ, ph, opt)
if ok && op.Type() == typ {
// Carry over the attempt count when doing an operation of the
// same type. The old operation exists and was canceled.
op2.attemptCount = op.AttemptCount() // carry the count
}
logger.Debugf("'%s' on cid '%s' has been created with phase '%s'", typ, pin.Cid, ph)
opt.operations[pin.Cid] = op2
opt.recordMetricUnsafe(op2, 1)
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
return op2
}
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
// Clean deletes an operation from the tracker if it is the one we are tracking
// (compares pointers).
func (opt *OperationTracker) Clean(ctx context.Context, op *Operation) {
opt.mu.Lock()
defer opt.mu.Unlock()
op2, ok := opt.operations[op.Cid()]
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
if ok && op == op2 { // same pointer
delete(opt.operations, op.Cid())
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
}
}
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
// Status returns the TrackerStatus associated to the last operation known
// with the given Cid. It returns false if we are not tracking any operation
// for the given Cid.
func (opt *OperationTracker) Status(ctx context.Context, c api.Cid) (api.TrackerStatus, bool) {
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
opt.mu.RLock()
defer opt.mu.RUnlock()
op, ok := opt.operations[c]
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
if !ok {
return 0, false
}
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
return op.ToTrackerStatus(), true
}
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
// SetError transitions an operation for a Cid into PhaseError if its Status
// is PhaseDone. Any other phases are considered in-flight and not touched.
// For things already in error, the error message is updated.
// Remote pins are ignored too.
// Only used in tests right now.
func (opt *OperationTracker) SetError(ctx context.Context, c api.Cid, err error) {
opt.mu.Lock()
defer opt.mu.Unlock()
op, ok := opt.operations[c]
if !ok {
return
}
if ty := op.Type(); ty == OperationRemote {
return
}
if ph := op.Phase(); ph == PhaseDone || ph == PhaseError {
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
op.SetPhase(PhaseError)
op.SetError(err)
}
}
func (opt *OperationTracker) unsafePinInfo(ctx context.Context, op *Operation, ipfs api.IPFSID) api.PinInfo {
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
if op == nil {
return api.PinInfo{
Cid: api.CidUndef,
Name: "",
Peer: opt.pid,
Origins: nil,
//Created: 0,
Metadata: nil,
PinInfoShort: api.PinInfoShort{
PeerName: opt.peerName,
IPFS: "",
Status: api.TrackerStatusUnpinned,
TS: time.Now(),
AttemptCount: 0,
PriorityPin: false,
Error: "",
},
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
}
}
return api.PinInfo{
Cid: op.Cid(),
Name: op.Pin().Name,
Peer: opt.pid,
Allocations: op.Pin().Allocations,
Origins: op.Pin().Origins,
Created: op.Pin().Timestamp,
Metadata: op.Pin().Metadata,
PinInfoShort: api.PinInfoShort{
PeerName: opt.peerName,
IPFS: ipfs.ID,
IPFSAddresses: ipfs.Addresses,
Status: op.ToTrackerStatus(),
TS: op.Timestamp(),
AttemptCount: op.AttemptCount(),
PriorityPin: op.PriorityPin(),
Error: op.Error(),
},
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
}
}
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
// Get returns a PinInfo object for Cid.
func (opt *OperationTracker) Get(ctx context.Context, c api.Cid, ipfs api.IPFSID) api.PinInfo {
ctx, span := trace.StartSpan(ctx, "optracker/GetAll")
defer span.End()
opt.mu.RLock()
defer opt.mu.RUnlock()
op := opt.operations[c]
pInfo := opt.unsafePinInfo(ctx, op, ipfs)
if !pInfo.Cid.Defined() {
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
pInfo.Cid = c
}
return pInfo
}
// GetExists returns a PinInfo object for a Cid only if there exists
// an associated Operation.
func (opt *OperationTracker) GetExists(ctx context.Context, c api.Cid, ipfs api.IPFSID) (api.PinInfo, bool) {
ctx, span := trace.StartSpan(ctx, "optracker/GetExists")
defer span.End()
opt.mu.RLock()
defer opt.mu.RUnlock()
op, ok := opt.operations[c]
if !ok {
return api.PinInfo{}, false
}
pInfo := opt.unsafePinInfo(ctx, op, ipfs)
return pInfo, true
}
// GetAll returns PinInfo objects for all known operations.
func (opt *OperationTracker) GetAll(ctx context.Context, ipfs api.IPFSID) []api.PinInfo {
ctx, span := trace.StartSpan(ctx, "optracker/GetAll")
defer span.End()
ch := make(chan api.PinInfo, 1024)
var pinfos []api.PinInfo
go opt.GetAllChannel(ctx, api.TrackerStatusUndefined, ipfs, ch)
for pinfo := range ch {
pinfos = append(pinfos, pinfo)
}
return pinfos
}
// GetAllChannel returns all known operations that match the filter on the
// provided channel. Blocks until done.
func (opt *OperationTracker) GetAllChannel(ctx context.Context, filter api.TrackerStatus, ipfs api.IPFSID, out chan<- api.PinInfo) error {
defer close(out)
opt.mu.RLock()
defer opt.mu.RUnlock()
for _, op := range opt.operations {
pinfo := opt.unsafePinInfo(ctx, op, ipfs)
if pinfo.Status.Match(filter) {
select {
case <-ctx.Done():
return fmt.Errorf("listing operations aborted: %w", ctx.Err())
default:
}
select {
case <-ctx.Done():
return fmt.Errorf("listing operations aborted: %w", ctx.Err())
case out <- pinfo:
}
}
}
return nil
}
// CleanAllDone deletes any operation from the tracker that is in PhaseDone.
func (opt *OperationTracker) CleanAllDone(ctx context.Context) {
opt.mu.Lock()
defer opt.mu.Unlock()
for _, op := range opt.operations {
if op.Phase() == PhaseDone {
delete(opt.operations, op.Cid())
}
}
}
// OpContext gets the context of an operation, if any.
func (opt *OperationTracker) OpContext(ctx context.Context, c api.Cid) context.Context {
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
opt.mu.RLock()
defer opt.mu.RUnlock()
op, ok := opt.operations[c]
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
if !ok {
return nil
}
Fix: maptracker race issues This commit attempts to fix race issues in the maptracker since the introduction of the OperationTracker. There were two main problems: * Duplicity tracking the state both in the state map and the opTracker * Non atomiciy of operations with different threads being able to affect other threads operations. A test performing random Track/Untracks on the same Cid quickly showed that items would sometimes stay as pin_queued or pin_unqueued. That happened because operations could be cancelled under the hood by a different request, while leaving the map status untouched. It was not simply to deal with this issues without a refactoring. First, the state map has been removed, and the operation tracker now provides status information for any Cid. This implies that the tracker keeps all operations and operations have a `PhaseDone`. There's also a new `OperationRemote` type. Secondly, operations are only created in the tracker and can only be removed by their creators (they can be overwritten by other operations though). Operations cannot be accessed directly and modifications are limited to setting Error for PhaseDone operations. After created, *Operations are queued in the pinWorker queues which handle any status updates. This means, that, even when an operation has been removed from the tracker, status updates will not interfere with any other newer operations. In the maptracker, only the Unpin worker Cleans operations once processed. A sucessful unpin is the only way that a delete() happens in the tracker map. Otherwise, operations stay there until a newer operation for the Cid arrives and 1) cancels the existing one 2) takes its place. The tracker refuses to create a new operation if a similar "ongoing" operation of the same type exists. The final change is that Recover and RecoverAll() are not async and play by the same rules as Track() and Untrack(), queueing the items to be recovered. Note: for stateless pintracker, the tracker will need to Clean() operation of type OperationPin as well, and complement the Status reported by the tracker with those coming from IPFS. License: MIT Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-25 16:32:10 +00:00
return op.Context()
}
// Filter returns a slice of api.PinInfos that had associated
// Operations that matched the provided filter. Note, only supports
// filters of type OperationType or Phase, any other type
// will result in a nil slice being returned.
func (opt *OperationTracker) Filter(ctx context.Context, ipfs api.IPFSID, filters ...interface{}) []api.PinInfo {
var pinfos []api.PinInfo
opt.mu.RLock()
defer opt.mu.RUnlock()
ops := filterOpsMap(ctx, opt.operations, filters)
for _, op := range ops {
pinfo := opt.unsafePinInfo(ctx, op, ipfs)
pinfos = append(pinfos, pinfo)
}
return pinfos
}
// filterOps returns a slice that only contains operations
// with the matching filter. Note, only supports
// filters of type OperationType or Phase, any other type
// will result in a nil slice being returned.
// Only used in tests right now.
func (opt *OperationTracker) filterOps(ctx context.Context, filters ...interface{}) []*Operation {
var fltops []*Operation
opt.mu.RLock()
defer opt.mu.RUnlock()
for _, op := range filterOpsMap(ctx, opt.operations, filters) {
fltops = append(fltops, op)
}
return fltops
}
func filterOpsMap(ctx context.Context, ops map[api.Cid]*Operation, filters []interface{}) map[api.Cid]*Operation {
fltops := make(map[api.Cid]*Operation)
if len(filters) < 1 {
return nil
}
if len(filters) == 1 {
filter(ctx, ops, fltops, filters[0])
return fltops
}
mainFilter, filters := filters[0], filters[1:]
filter(ctx, ops, fltops, mainFilter)
return filterOpsMap(ctx, fltops, filters)
}
func filter(ctx context.Context, in, out map[api.Cid]*Operation, filter interface{}) {
for _, op := range in {
switch filter.(type) {
case OperationType:
if op.Type() == filter {
out[op.Cid()] = op
}
case Phase:
if op.Phase() == filter {
out[op.Cid()] = op
}
}
}
}
func initializeMetrics(ctx context.Context) {
stats.Record(ctx, observations.PinsPinError.M(0))
stats.Record(ctx, observations.PinsQueued.M(0))
stats.Record(ctx, observations.PinsPinning.M(0))
}
func (opt *OperationTracker) recordMetricUnsafe(op *Operation, val int64) {
if opt == nil || op == nil {
return
}
if op.opType == OperationPin {
switch op.phase {
case PhaseError:
pinErrors := atomic.AddInt64(&opt.pinErrorCount, val)
stats.Record(op.Context(), observations.PinsPinError.M(pinErrors))
case PhaseQueued:
pinQueued := atomic.AddInt64(&opt.pinQueuedCount, val)
stats.Record(op.Context(), observations.PinsQueued.M(pinQueued))
case PhaseInProgress:
pinning := atomic.AddInt64(&opt.pinningCount, val)
stats.Record(op.Context(), observations.PinsPinning.M(pinning))
case PhaseDone:
// we have no metric to log anything
}
}
}
func (opt *OperationTracker) recordMetric(op *Operation, val int64) {
if op == nil {
return
}
op.mu.RLock()
{
opt.recordMetricUnsafe(op, val)
}
op.mu.RUnlock()
}
// PinQueueSize returns the current number of items queued to pin.
func (opt *OperationTracker) PinQueueSize() int64 {
return atomic.LoadInt64(&opt.pinQueuedCount)
}