coder/provisionerd/provisionerd_test.go

1167 lines
37 KiB
Go
Raw Normal View History

package provisionerd_test
import (
"archive/tar"
"bytes"
"context"
"fmt"
"io"
"os"
"path/filepath"
"sync"
"testing"
"time"
"github.com/hashicorp/yamux"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"go.uber.org/atomic"
"go.uber.org/goleak"
"golang.org/x/xerrors"
"storj.io/drpc/drpcmux"
"storj.io/drpc/drpcserver"
"cdr.dev/slog"
"cdr.dev/slog/sloggers/slogtest"
"github.com/coder/coder/provisionerd"
"github.com/coder/coder/provisionerd/proto"
"github.com/coder/coder/provisionerd/runner"
"github.com/coder/coder/provisionersdk"
sdkproto "github.com/coder/coder/provisionersdk/proto"
"github.com/coder/coder/testutil"
)
func TestMain(m *testing.M) {
goleak.VerifyTestMain(m)
}
func closedWithin(c chan struct{}, d time.Duration) func() bool {
return func() bool {
select {
case <-c:
return true
case <-time.After(d):
return false
}
}
}
func TestProvisionerd(t *testing.T) {
t.Parallel()
noopUpdateJob := func(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error) {
return &proto.UpdateJobResponse{}, nil
}
t.Run("InstantClose", func(t *testing.T) {
t.Parallel()
closer := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{}), nil
}, provisionerd.Provisioners{})
require.NoError(t, closer.Close())
})
t.Run("ConnectErrorClose", func(t *testing.T) {
t.Parallel()
completeChan := make(chan struct{})
closer := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
defer close(completeChan)
return nil, xerrors.New("an error")
}, provisionerd.Provisioners{})
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, closer.Close())
})
t.Run("AcquireEmptyJob", func(t *testing.T) {
// The provisioner daemon is supposed to skip the job acquire if
// the job provided is empty. This is to show it successfully
// tried to get a job, but none were available.
t.Parallel()
completeChan := make(chan struct{})
closer := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
acquireJobAttempt := 0
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
if acquireJobAttempt == 1 {
close(completeChan)
}
acquireJobAttempt++
return &proto.AcquiredJob{}, nil
},
updateJob: noopUpdateJob,
}), nil
}, provisionerd.Provisioners{})
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, closer.Close())
})
t.Run("CloseCancelsJob", func(t *testing.T) {
t.Parallel()
completeChan := make(chan struct{})
var completed sync.Once
var closer io.Closer
var closerMutex sync.Mutex
closerMutex.Lock()
closer = createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
"test.txt": "content",
}),
Type: &proto.AcquiredJob_TemplateImport_{
TemplateImport: &proto.AcquiredJob_TemplateImport{
Metadata: &sdkproto.Provision_Metadata{},
},
},
}, nil
},
updateJob: noopUpdateJob,
failJob: func(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error) {
completed.Do(func() {
close(completeChan)
})
return &proto.Empty{}, nil
},
}), nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{
parse: func(request *sdkproto.Parse_Request, stream sdkproto.DRPCProvisioner_ParseStream) error {
closerMutex.Lock()
defer closerMutex.Unlock()
return closer.Close()
},
}),
})
closerMutex.Unlock()
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, closer.Close())
})
t.Run("MaliciousTar", func(t *testing.T) {
// Ensures tars with "../../../etc/passwd" as the path
// are not allowed to run, and will fail the job.
t.Parallel()
var (
completeChan = make(chan struct{})
completeOnce sync.Once
)
closer := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
"../../../etc/passwd": "content",
}),
Type: &proto.AcquiredJob_TemplateImport_{
TemplateImport: &proto.AcquiredJob_TemplateImport{
Metadata: &sdkproto.Provision_Metadata{},
},
},
}, nil
},
updateJob: noopUpdateJob,
failJob: func(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error) {
completeOnce.Do(func() { close(completeChan) })
return &proto.Empty{}, nil
},
}), nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{}),
})
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, closer.Close())
})
feat: Add provisionerdaemon to coderd (#141) * feat: Add history middleware parameters These will be used for streaming logs, checking status, and other operations related to workspace and project history. * refactor: Move all HTTP routes to top-level struct Nesting all structs behind their respective structures is leaky, and promotes naming conflicts between handlers. Our HTTP routes cannot have conflicts, so neither should function naming. * Add provisioner daemon routes * Add periodic updates * Skip pubsub if short * Return jobs with WorkspaceHistory * Add endpoints for extracting singular history * The full end-to-end operation works * fix: Disable compression for websocket dRPC transport (#145) There is a race condition in the interop between the websocket and `dRPC`: https://github.com/coder/coder/runs/5038545709?check_suite_focus=true#step:7:117 - it seems both the websocket and dRPC feel like they own the `byte[]` being sent between them. This can lead to data races, in which both `dRPC` and the websocket are writing. This is just tracking some experimentation to fix that race condition ## Run results: ## - Run 1: peer test failure - Run 2: peer test failure - Run 3: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040858460?check_suite_focus=true#step:8:45 ``` status code 412: The provided project history is running. Wait for it to complete importing!` ``` - Run 4: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040957999?check_suite_focus=true#step:7:176 ``` workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` - Run 5: peer failure - Run 6: Pass ✅ - Run 7: Peer failure ## Open Questions: ## ### Is `dRPC` or `websocket` at fault for the data race? It looks like this condition is specifically happening when `dRPC` decides to [`SendError`]). This constructs a new byte payload from [`MarshalError`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/error.go#L15) - so `dRPC` has created this buffer and owns it. From `dRPC`'s perspective, the callstack looks like this: - [`sendPacket`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcstream/stream.go#L253) - [`writeFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L65) - [`AppendFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/packet.go#L128) - with finally the data race happening here: ```go // AppendFrame appends a marshaled form of the frame to the provided buffer. func AppendFrame(buf []byte, fr Frame) []byte { ... out := buf out = append(out, control). // <--------- ``` This should be fine, since `dPRC` create this buffer, and is taking the byte buffer constructed from `MarshalError` and tacking a bunch of headers on it to create a proper frame. Once `dRPC` is done writing, it _hangs onto the buffer and resets it here__: https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L73 However... the websocket implementation, once it gets the buffer, it runs a `statelessDeflate` [here](https://github.com/nhooyr/websocket/blob/8dee580a7f74cf1713400307b4eee514b927870f/write.go#L180), which compresses the buffer on the fly. This functionality actually [mutates the buffer in place](https://github.com/klauspost/compress/blob/a1a9cfc821f00faf2f5231beaa96244344d50391/flate/stateless.go#L94), which is where get our race. In the case where the `byte[]` aren't being manipulated anywhere else, this compress-in-place operation would be safe, and that's probably the case for most over-the-wire usages. In this case, though, where we're plumbing `dRPC` -> websocket, they both are manipulating it (`dRPC` is reusing the buffer for the next `write`, and `websocket` is compressing on the fly). ### Why does cloning on `Read` fail? Get a bunch of errors like: ``` 2022/02/02 19:26:10 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 ``` # UPDATE: We decided we could disable websocket compression, which would avoid the race because the in-place `deflate` operaton would no longer be run. Trying that out now: - Run 1: ✅ - Run 2: https://github.com/coder/coder/runs/5042645522?check_suite_focus=true#step:8:338 - Run 3: ✅ - Run 4: https://github.com/coder/coder/runs/5042988758?check_suite_focus=true#step:7:168 - Run 5: ✅ * fix: Remove race condition with acquiredJobDone channel (#148) Found another data race while running the tests: https://github.com/coder/coder/runs/5044320845?check_suite_focus=true#step:7:83 __Issue:__ There is a race in the p.acquiredJobDone chan - in particular, there can be a case where we're waiting on the channel to finish (in close) with <-p.acquiredJobDone, but in parallel, an acquireJob could've been started, which would create a new channel for p.acquiredJobDone. There is a similar race in `close(..)`ing the channel, which also came up in test runs. __Fix:__ Instead of recreating the channel everytime, we can use `sync.WaitGroup` to accomplish the same functionality - a semaphore to make close wait for the current job to wrap up. * fix: Bump up workspace history timeout (#149) This is an attempted fix for failures like: https://github.com/coder/coder/runs/5043435263?check_suite_focus=true#step:7:32 Looking at the timing of the test: ``` t.go:56: 2022-02-02 21:33:21.964 [DEBUG] (terraform-provisioner) <provision.go:139> ran apply t.go:56: 2022-02-02 21:33:21.991 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.050 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.090 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.140 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.195 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.240 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` It appears that the `terraform apply` job had just finished - with less than a second to spare until our `require.Eventually` completes - but there's still work to be done (ie, collecting the state files). So my suspicion is that terraform might, in some cases, exceed our 5s timeout. Note that in the setup for this test - there is a similar project history wait that waits for 15s, so I borrowed that here. In the future - we can look at potentially using a simple echo provider to exercise this in the unit test, in a way that is more reliable in terms of timing. I'll log an issue to track that. Co-authored-by: Bryan <bryan@coder.com>
2022-02-03 20:34:50 +00:00
t.Run("RunningPeriodicUpdate", func(t *testing.T) {
t.Parallel()
var (
completeChan = make(chan struct{})
completeOnce sync.Once
)
feat: Add provisionerdaemon to coderd (#141) * feat: Add history middleware parameters These will be used for streaming logs, checking status, and other operations related to workspace and project history. * refactor: Move all HTTP routes to top-level struct Nesting all structs behind their respective structures is leaky, and promotes naming conflicts between handlers. Our HTTP routes cannot have conflicts, so neither should function naming. * Add provisioner daemon routes * Add periodic updates * Skip pubsub if short * Return jobs with WorkspaceHistory * Add endpoints for extracting singular history * The full end-to-end operation works * fix: Disable compression for websocket dRPC transport (#145) There is a race condition in the interop between the websocket and `dRPC`: https://github.com/coder/coder/runs/5038545709?check_suite_focus=true#step:7:117 - it seems both the websocket and dRPC feel like they own the `byte[]` being sent between them. This can lead to data races, in which both `dRPC` and the websocket are writing. This is just tracking some experimentation to fix that race condition ## Run results: ## - Run 1: peer test failure - Run 2: peer test failure - Run 3: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040858460?check_suite_focus=true#step:8:45 ``` status code 412: The provided project history is running. Wait for it to complete importing!` ``` - Run 4: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040957999?check_suite_focus=true#step:7:176 ``` workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` - Run 5: peer failure - Run 6: Pass ✅ - Run 7: Peer failure ## Open Questions: ## ### Is `dRPC` or `websocket` at fault for the data race? It looks like this condition is specifically happening when `dRPC` decides to [`SendError`]). This constructs a new byte payload from [`MarshalError`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/error.go#L15) - so `dRPC` has created this buffer and owns it. From `dRPC`'s perspective, the callstack looks like this: - [`sendPacket`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcstream/stream.go#L253) - [`writeFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L65) - [`AppendFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/packet.go#L128) - with finally the data race happening here: ```go // AppendFrame appends a marshaled form of the frame to the provided buffer. func AppendFrame(buf []byte, fr Frame) []byte { ... out := buf out = append(out, control). // <--------- ``` This should be fine, since `dPRC` create this buffer, and is taking the byte buffer constructed from `MarshalError` and tacking a bunch of headers on it to create a proper frame. Once `dRPC` is done writing, it _hangs onto the buffer and resets it here__: https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L73 However... the websocket implementation, once it gets the buffer, it runs a `statelessDeflate` [here](https://github.com/nhooyr/websocket/blob/8dee580a7f74cf1713400307b4eee514b927870f/write.go#L180), which compresses the buffer on the fly. This functionality actually [mutates the buffer in place](https://github.com/klauspost/compress/blob/a1a9cfc821f00faf2f5231beaa96244344d50391/flate/stateless.go#L94), which is where get our race. In the case where the `byte[]` aren't being manipulated anywhere else, this compress-in-place operation would be safe, and that's probably the case for most over-the-wire usages. In this case, though, where we're plumbing `dRPC` -> websocket, they both are manipulating it (`dRPC` is reusing the buffer for the next `write`, and `websocket` is compressing on the fly). ### Why does cloning on `Read` fail? Get a bunch of errors like: ``` 2022/02/02 19:26:10 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 ``` # UPDATE: We decided we could disable websocket compression, which would avoid the race because the in-place `deflate` operaton would no longer be run. Trying that out now: - Run 1: ✅ - Run 2: https://github.com/coder/coder/runs/5042645522?check_suite_focus=true#step:8:338 - Run 3: ✅ - Run 4: https://github.com/coder/coder/runs/5042988758?check_suite_focus=true#step:7:168 - Run 5: ✅ * fix: Remove race condition with acquiredJobDone channel (#148) Found another data race while running the tests: https://github.com/coder/coder/runs/5044320845?check_suite_focus=true#step:7:83 __Issue:__ There is a race in the p.acquiredJobDone chan - in particular, there can be a case where we're waiting on the channel to finish (in close) with <-p.acquiredJobDone, but in parallel, an acquireJob could've been started, which would create a new channel for p.acquiredJobDone. There is a similar race in `close(..)`ing the channel, which also came up in test runs. __Fix:__ Instead of recreating the channel everytime, we can use `sync.WaitGroup` to accomplish the same functionality - a semaphore to make close wait for the current job to wrap up. * fix: Bump up workspace history timeout (#149) This is an attempted fix for failures like: https://github.com/coder/coder/runs/5043435263?check_suite_focus=true#step:7:32 Looking at the timing of the test: ``` t.go:56: 2022-02-02 21:33:21.964 [DEBUG] (terraform-provisioner) <provision.go:139> ran apply t.go:56: 2022-02-02 21:33:21.991 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.050 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.090 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.140 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.195 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.240 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` It appears that the `terraform apply` job had just finished - with less than a second to spare until our `require.Eventually` completes - but there's still work to be done (ie, collecting the state files). So my suspicion is that terraform might, in some cases, exceed our 5s timeout. Note that in the setup for this test - there is a similar project history wait that waits for 15s, so I borrowed that here. In the future - we can look at potentially using a simple echo provider to exercise this in the unit test, in a way that is more reliable in terms of timing. I'll log an issue to track that. Co-authored-by: Bryan <bryan@coder.com>
2022-02-03 20:34:50 +00:00
closer := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
feat: Add provisionerdaemon to coderd (#141) * feat: Add history middleware parameters These will be used for streaming logs, checking status, and other operations related to workspace and project history. * refactor: Move all HTTP routes to top-level struct Nesting all structs behind their respective structures is leaky, and promotes naming conflicts between handlers. Our HTTP routes cannot have conflicts, so neither should function naming. * Add provisioner daemon routes * Add periodic updates * Skip pubsub if short * Return jobs with WorkspaceHistory * Add endpoints for extracting singular history * The full end-to-end operation works * fix: Disable compression for websocket dRPC transport (#145) There is a race condition in the interop between the websocket and `dRPC`: https://github.com/coder/coder/runs/5038545709?check_suite_focus=true#step:7:117 - it seems both the websocket and dRPC feel like they own the `byte[]` being sent between them. This can lead to data races, in which both `dRPC` and the websocket are writing. This is just tracking some experimentation to fix that race condition ## Run results: ## - Run 1: peer test failure - Run 2: peer test failure - Run 3: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040858460?check_suite_focus=true#step:8:45 ``` status code 412: The provided project history is running. Wait for it to complete importing!` ``` - Run 4: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040957999?check_suite_focus=true#step:7:176 ``` workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` - Run 5: peer failure - Run 6: Pass ✅ - Run 7: Peer failure ## Open Questions: ## ### Is `dRPC` or `websocket` at fault for the data race? It looks like this condition is specifically happening when `dRPC` decides to [`SendError`]). This constructs a new byte payload from [`MarshalError`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/error.go#L15) - so `dRPC` has created this buffer and owns it. From `dRPC`'s perspective, the callstack looks like this: - [`sendPacket`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcstream/stream.go#L253) - [`writeFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L65) - [`AppendFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/packet.go#L128) - with finally the data race happening here: ```go // AppendFrame appends a marshaled form of the frame to the provided buffer. func AppendFrame(buf []byte, fr Frame) []byte { ... out := buf out = append(out, control). // <--------- ``` This should be fine, since `dPRC` create this buffer, and is taking the byte buffer constructed from `MarshalError` and tacking a bunch of headers on it to create a proper frame. Once `dRPC` is done writing, it _hangs onto the buffer and resets it here__: https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L73 However... the websocket implementation, once it gets the buffer, it runs a `statelessDeflate` [here](https://github.com/nhooyr/websocket/blob/8dee580a7f74cf1713400307b4eee514b927870f/write.go#L180), which compresses the buffer on the fly. This functionality actually [mutates the buffer in place](https://github.com/klauspost/compress/blob/a1a9cfc821f00faf2f5231beaa96244344d50391/flate/stateless.go#L94), which is where get our race. In the case where the `byte[]` aren't being manipulated anywhere else, this compress-in-place operation would be safe, and that's probably the case for most over-the-wire usages. In this case, though, where we're plumbing `dRPC` -> websocket, they both are manipulating it (`dRPC` is reusing the buffer for the next `write`, and `websocket` is compressing on the fly). ### Why does cloning on `Read` fail? Get a bunch of errors like: ``` 2022/02/02 19:26:10 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 ``` # UPDATE: We decided we could disable websocket compression, which would avoid the race because the in-place `deflate` operaton would no longer be run. Trying that out now: - Run 1: ✅ - Run 2: https://github.com/coder/coder/runs/5042645522?check_suite_focus=true#step:8:338 - Run 3: ✅ - Run 4: https://github.com/coder/coder/runs/5042988758?check_suite_focus=true#step:7:168 - Run 5: ✅ * fix: Remove race condition with acquiredJobDone channel (#148) Found another data race while running the tests: https://github.com/coder/coder/runs/5044320845?check_suite_focus=true#step:7:83 __Issue:__ There is a race in the p.acquiredJobDone chan - in particular, there can be a case where we're waiting on the channel to finish (in close) with <-p.acquiredJobDone, but in parallel, an acquireJob could've been started, which would create a new channel for p.acquiredJobDone. There is a similar race in `close(..)`ing the channel, which also came up in test runs. __Fix:__ Instead of recreating the channel everytime, we can use `sync.WaitGroup` to accomplish the same functionality - a semaphore to make close wait for the current job to wrap up. * fix: Bump up workspace history timeout (#149) This is an attempted fix for failures like: https://github.com/coder/coder/runs/5043435263?check_suite_focus=true#step:7:32 Looking at the timing of the test: ``` t.go:56: 2022-02-02 21:33:21.964 [DEBUG] (terraform-provisioner) <provision.go:139> ran apply t.go:56: 2022-02-02 21:33:21.991 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.050 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.090 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.140 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.195 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.240 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` It appears that the `terraform apply` job had just finished - with less than a second to spare until our `require.Eventually` completes - but there's still work to be done (ie, collecting the state files). So my suspicion is that terraform might, in some cases, exceed our 5s timeout. Note that in the setup for this test - there is a similar project history wait that waits for 15s, so I borrowed that here. In the future - we can look at potentially using a simple echo provider to exercise this in the unit test, in a way that is more reliable in terms of timing. I'll log an issue to track that. Co-authored-by: Bryan <bryan@coder.com>
2022-02-03 20:34:50 +00:00
"test.txt": "content",
}),
Type: &proto.AcquiredJob_TemplateImport_{
TemplateImport: &proto.AcquiredJob_TemplateImport{
Metadata: &sdkproto.Provision_Metadata{},
},
feat: Add provisionerdaemon to coderd (#141) * feat: Add history middleware parameters These will be used for streaming logs, checking status, and other operations related to workspace and project history. * refactor: Move all HTTP routes to top-level struct Nesting all structs behind their respective structures is leaky, and promotes naming conflicts between handlers. Our HTTP routes cannot have conflicts, so neither should function naming. * Add provisioner daemon routes * Add periodic updates * Skip pubsub if short * Return jobs with WorkspaceHistory * Add endpoints for extracting singular history * The full end-to-end operation works * fix: Disable compression for websocket dRPC transport (#145) There is a race condition in the interop between the websocket and `dRPC`: https://github.com/coder/coder/runs/5038545709?check_suite_focus=true#step:7:117 - it seems both the websocket and dRPC feel like they own the `byte[]` being sent between them. This can lead to data races, in which both `dRPC` and the websocket are writing. This is just tracking some experimentation to fix that race condition ## Run results: ## - Run 1: peer test failure - Run 2: peer test failure - Run 3: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040858460?check_suite_focus=true#step:8:45 ``` status code 412: The provided project history is running. Wait for it to complete importing!` ``` - Run 4: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040957999?check_suite_focus=true#step:7:176 ``` workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` - Run 5: peer failure - Run 6: Pass ✅ - Run 7: Peer failure ## Open Questions: ## ### Is `dRPC` or `websocket` at fault for the data race? It looks like this condition is specifically happening when `dRPC` decides to [`SendError`]). This constructs a new byte payload from [`MarshalError`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/error.go#L15) - so `dRPC` has created this buffer and owns it. From `dRPC`'s perspective, the callstack looks like this: - [`sendPacket`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcstream/stream.go#L253) - [`writeFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L65) - [`AppendFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/packet.go#L128) - with finally the data race happening here: ```go // AppendFrame appends a marshaled form of the frame to the provided buffer. func AppendFrame(buf []byte, fr Frame) []byte { ... out := buf out = append(out, control). // <--------- ``` This should be fine, since `dPRC` create this buffer, and is taking the byte buffer constructed from `MarshalError` and tacking a bunch of headers on it to create a proper frame. Once `dRPC` is done writing, it _hangs onto the buffer and resets it here__: https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L73 However... the websocket implementation, once it gets the buffer, it runs a `statelessDeflate` [here](https://github.com/nhooyr/websocket/blob/8dee580a7f74cf1713400307b4eee514b927870f/write.go#L180), which compresses the buffer on the fly. This functionality actually [mutates the buffer in place](https://github.com/klauspost/compress/blob/a1a9cfc821f00faf2f5231beaa96244344d50391/flate/stateless.go#L94), which is where get our race. In the case where the `byte[]` aren't being manipulated anywhere else, this compress-in-place operation would be safe, and that's probably the case for most over-the-wire usages. In this case, though, where we're plumbing `dRPC` -> websocket, they both are manipulating it (`dRPC` is reusing the buffer for the next `write`, and `websocket` is compressing on the fly). ### Why does cloning on `Read` fail? Get a bunch of errors like: ``` 2022/02/02 19:26:10 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 ``` # UPDATE: We decided we could disable websocket compression, which would avoid the race because the in-place `deflate` operaton would no longer be run. Trying that out now: - Run 1: ✅ - Run 2: https://github.com/coder/coder/runs/5042645522?check_suite_focus=true#step:8:338 - Run 3: ✅ - Run 4: https://github.com/coder/coder/runs/5042988758?check_suite_focus=true#step:7:168 - Run 5: ✅ * fix: Remove race condition with acquiredJobDone channel (#148) Found another data race while running the tests: https://github.com/coder/coder/runs/5044320845?check_suite_focus=true#step:7:83 __Issue:__ There is a race in the p.acquiredJobDone chan - in particular, there can be a case where we're waiting on the channel to finish (in close) with <-p.acquiredJobDone, but in parallel, an acquireJob could've been started, which would create a new channel for p.acquiredJobDone. There is a similar race in `close(..)`ing the channel, which also came up in test runs. __Fix:__ Instead of recreating the channel everytime, we can use `sync.WaitGroup` to accomplish the same functionality - a semaphore to make close wait for the current job to wrap up. * fix: Bump up workspace history timeout (#149) This is an attempted fix for failures like: https://github.com/coder/coder/runs/5043435263?check_suite_focus=true#step:7:32 Looking at the timing of the test: ``` t.go:56: 2022-02-02 21:33:21.964 [DEBUG] (terraform-provisioner) <provision.go:139> ran apply t.go:56: 2022-02-02 21:33:21.991 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.050 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.090 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.140 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.195 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.240 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` It appears that the `terraform apply` job had just finished - with less than a second to spare until our `require.Eventually` completes - but there's still work to be done (ie, collecting the state files). So my suspicion is that terraform might, in some cases, exceed our 5s timeout. Note that in the setup for this test - there is a similar project history wait that waits for 15s, so I borrowed that here. In the future - we can look at potentially using a simple echo provider to exercise this in the unit test, in a way that is more reliable in terms of timing. I'll log an issue to track that. Co-authored-by: Bryan <bryan@coder.com>
2022-02-03 20:34:50 +00:00
},
}, nil
},
updateJob: func(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error) {
completeOnce.Do(func() { close(completeChan) })
return &proto.UpdateJobResponse{}, nil
feat: Add provisionerdaemon to coderd (#141) * feat: Add history middleware parameters These will be used for streaming logs, checking status, and other operations related to workspace and project history. * refactor: Move all HTTP routes to top-level struct Nesting all structs behind their respective structures is leaky, and promotes naming conflicts between handlers. Our HTTP routes cannot have conflicts, so neither should function naming. * Add provisioner daemon routes * Add periodic updates * Skip pubsub if short * Return jobs with WorkspaceHistory * Add endpoints for extracting singular history * The full end-to-end operation works * fix: Disable compression for websocket dRPC transport (#145) There is a race condition in the interop between the websocket and `dRPC`: https://github.com/coder/coder/runs/5038545709?check_suite_focus=true#step:7:117 - it seems both the websocket and dRPC feel like they own the `byte[]` being sent between them. This can lead to data races, in which both `dRPC` and the websocket are writing. This is just tracking some experimentation to fix that race condition ## Run results: ## - Run 1: peer test failure - Run 2: peer test failure - Run 3: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040858460?check_suite_focus=true#step:8:45 ``` status code 412: The provided project history is running. Wait for it to complete importing!` ``` - Run 4: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040957999?check_suite_focus=true#step:7:176 ``` workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` - Run 5: peer failure - Run 6: Pass ✅ - Run 7: Peer failure ## Open Questions: ## ### Is `dRPC` or `websocket` at fault for the data race? It looks like this condition is specifically happening when `dRPC` decides to [`SendError`]). This constructs a new byte payload from [`MarshalError`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/error.go#L15) - so `dRPC` has created this buffer and owns it. From `dRPC`'s perspective, the callstack looks like this: - [`sendPacket`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcstream/stream.go#L253) - [`writeFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L65) - [`AppendFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/packet.go#L128) - with finally the data race happening here: ```go // AppendFrame appends a marshaled form of the frame to the provided buffer. func AppendFrame(buf []byte, fr Frame) []byte { ... out := buf out = append(out, control). // <--------- ``` This should be fine, since `dPRC` create this buffer, and is taking the byte buffer constructed from `MarshalError` and tacking a bunch of headers on it to create a proper frame. Once `dRPC` is done writing, it _hangs onto the buffer and resets it here__: https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L73 However... the websocket implementation, once it gets the buffer, it runs a `statelessDeflate` [here](https://github.com/nhooyr/websocket/blob/8dee580a7f74cf1713400307b4eee514b927870f/write.go#L180), which compresses the buffer on the fly. This functionality actually [mutates the buffer in place](https://github.com/klauspost/compress/blob/a1a9cfc821f00faf2f5231beaa96244344d50391/flate/stateless.go#L94), which is where get our race. In the case where the `byte[]` aren't being manipulated anywhere else, this compress-in-place operation would be safe, and that's probably the case for most over-the-wire usages. In this case, though, where we're plumbing `dRPC` -> websocket, they both are manipulating it (`dRPC` is reusing the buffer for the next `write`, and `websocket` is compressing on the fly). ### Why does cloning on `Read` fail? Get a bunch of errors like: ``` 2022/02/02 19:26:10 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 ``` # UPDATE: We decided we could disable websocket compression, which would avoid the race because the in-place `deflate` operaton would no longer be run. Trying that out now: - Run 1: ✅ - Run 2: https://github.com/coder/coder/runs/5042645522?check_suite_focus=true#step:8:338 - Run 3: ✅ - Run 4: https://github.com/coder/coder/runs/5042988758?check_suite_focus=true#step:7:168 - Run 5: ✅ * fix: Remove race condition with acquiredJobDone channel (#148) Found another data race while running the tests: https://github.com/coder/coder/runs/5044320845?check_suite_focus=true#step:7:83 __Issue:__ There is a race in the p.acquiredJobDone chan - in particular, there can be a case where we're waiting on the channel to finish (in close) with <-p.acquiredJobDone, but in parallel, an acquireJob could've been started, which would create a new channel for p.acquiredJobDone. There is a similar race in `close(..)`ing the channel, which also came up in test runs. __Fix:__ Instead of recreating the channel everytime, we can use `sync.WaitGroup` to accomplish the same functionality - a semaphore to make close wait for the current job to wrap up. * fix: Bump up workspace history timeout (#149) This is an attempted fix for failures like: https://github.com/coder/coder/runs/5043435263?check_suite_focus=true#step:7:32 Looking at the timing of the test: ``` t.go:56: 2022-02-02 21:33:21.964 [DEBUG] (terraform-provisioner) <provision.go:139> ran apply t.go:56: 2022-02-02 21:33:21.991 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.050 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.090 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.140 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.195 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.240 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` It appears that the `terraform apply` job had just finished - with less than a second to spare until our `require.Eventually` completes - but there's still work to be done (ie, collecting the state files). So my suspicion is that terraform might, in some cases, exceed our 5s timeout. Note that in the setup for this test - there is a similar project history wait that waits for 15s, so I borrowed that here. In the future - we can look at potentially using a simple echo provider to exercise this in the unit test, in a way that is more reliable in terms of timing. I'll log an issue to track that. Co-authored-by: Bryan <bryan@coder.com>
2022-02-03 20:34:50 +00:00
},
failJob: func(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error) {
feat: Add provisionerdaemon to coderd (#141) * feat: Add history middleware parameters These will be used for streaming logs, checking status, and other operations related to workspace and project history. * refactor: Move all HTTP routes to top-level struct Nesting all structs behind their respective structures is leaky, and promotes naming conflicts between handlers. Our HTTP routes cannot have conflicts, so neither should function naming. * Add provisioner daemon routes * Add periodic updates * Skip pubsub if short * Return jobs with WorkspaceHistory * Add endpoints for extracting singular history * The full end-to-end operation works * fix: Disable compression for websocket dRPC transport (#145) There is a race condition in the interop between the websocket and `dRPC`: https://github.com/coder/coder/runs/5038545709?check_suite_focus=true#step:7:117 - it seems both the websocket and dRPC feel like they own the `byte[]` being sent between them. This can lead to data races, in which both `dRPC` and the websocket are writing. This is just tracking some experimentation to fix that race condition ## Run results: ## - Run 1: peer test failure - Run 2: peer test failure - Run 3: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040858460?check_suite_focus=true#step:8:45 ``` status code 412: The provided project history is running. Wait for it to complete importing!` ``` - Run 4: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040957999?check_suite_focus=true#step:7:176 ``` workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` - Run 5: peer failure - Run 6: Pass ✅ - Run 7: Peer failure ## Open Questions: ## ### Is `dRPC` or `websocket` at fault for the data race? It looks like this condition is specifically happening when `dRPC` decides to [`SendError`]). This constructs a new byte payload from [`MarshalError`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/error.go#L15) - so `dRPC` has created this buffer and owns it. From `dRPC`'s perspective, the callstack looks like this: - [`sendPacket`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcstream/stream.go#L253) - [`writeFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L65) - [`AppendFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/packet.go#L128) - with finally the data race happening here: ```go // AppendFrame appends a marshaled form of the frame to the provided buffer. func AppendFrame(buf []byte, fr Frame) []byte { ... out := buf out = append(out, control). // <--------- ``` This should be fine, since `dPRC` create this buffer, and is taking the byte buffer constructed from `MarshalError` and tacking a bunch of headers on it to create a proper frame. Once `dRPC` is done writing, it _hangs onto the buffer and resets it here__: https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L73 However... the websocket implementation, once it gets the buffer, it runs a `statelessDeflate` [here](https://github.com/nhooyr/websocket/blob/8dee580a7f74cf1713400307b4eee514b927870f/write.go#L180), which compresses the buffer on the fly. This functionality actually [mutates the buffer in place](https://github.com/klauspost/compress/blob/a1a9cfc821f00faf2f5231beaa96244344d50391/flate/stateless.go#L94), which is where get our race. In the case where the `byte[]` aren't being manipulated anywhere else, this compress-in-place operation would be safe, and that's probably the case for most over-the-wire usages. In this case, though, where we're plumbing `dRPC` -> websocket, they both are manipulating it (`dRPC` is reusing the buffer for the next `write`, and `websocket` is compressing on the fly). ### Why does cloning on `Read` fail? Get a bunch of errors like: ``` 2022/02/02 19:26:10 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 ``` # UPDATE: We decided we could disable websocket compression, which would avoid the race because the in-place `deflate` operaton would no longer be run. Trying that out now: - Run 1: ✅ - Run 2: https://github.com/coder/coder/runs/5042645522?check_suite_focus=true#step:8:338 - Run 3: ✅ - Run 4: https://github.com/coder/coder/runs/5042988758?check_suite_focus=true#step:7:168 - Run 5: ✅ * fix: Remove race condition with acquiredJobDone channel (#148) Found another data race while running the tests: https://github.com/coder/coder/runs/5044320845?check_suite_focus=true#step:7:83 __Issue:__ There is a race in the p.acquiredJobDone chan - in particular, there can be a case where we're waiting on the channel to finish (in close) with <-p.acquiredJobDone, but in parallel, an acquireJob could've been started, which would create a new channel for p.acquiredJobDone. There is a similar race in `close(..)`ing the channel, which also came up in test runs. __Fix:__ Instead of recreating the channel everytime, we can use `sync.WaitGroup` to accomplish the same functionality - a semaphore to make close wait for the current job to wrap up. * fix: Bump up workspace history timeout (#149) This is an attempted fix for failures like: https://github.com/coder/coder/runs/5043435263?check_suite_focus=true#step:7:32 Looking at the timing of the test: ``` t.go:56: 2022-02-02 21:33:21.964 [DEBUG] (terraform-provisioner) <provision.go:139> ran apply t.go:56: 2022-02-02 21:33:21.991 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.050 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.090 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.140 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.195 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.240 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` It appears that the `terraform apply` job had just finished - with less than a second to spare until our `require.Eventually` completes - but there's still work to be done (ie, collecting the state files). So my suspicion is that terraform might, in some cases, exceed our 5s timeout. Note that in the setup for this test - there is a similar project history wait that waits for 15s, so I borrowed that here. In the future - we can look at potentially using a simple echo provider to exercise this in the unit test, in a way that is more reliable in terms of timing. I'll log an issue to track that. Co-authored-by: Bryan <bryan@coder.com>
2022-02-03 20:34:50 +00:00
return &proto.Empty{}, nil
},
}), nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{
parse: func(request *sdkproto.Parse_Request, stream sdkproto.DRPCProvisioner_ParseStream) error {
<-stream.Context().Done()
return nil
},
}),
})
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
feat: Add provisionerdaemon to coderd (#141) * feat: Add history middleware parameters These will be used for streaming logs, checking status, and other operations related to workspace and project history. * refactor: Move all HTTP routes to top-level struct Nesting all structs behind their respective structures is leaky, and promotes naming conflicts between handlers. Our HTTP routes cannot have conflicts, so neither should function naming. * Add provisioner daemon routes * Add periodic updates * Skip pubsub if short * Return jobs with WorkspaceHistory * Add endpoints for extracting singular history * The full end-to-end operation works * fix: Disable compression for websocket dRPC transport (#145) There is a race condition in the interop between the websocket and `dRPC`: https://github.com/coder/coder/runs/5038545709?check_suite_focus=true#step:7:117 - it seems both the websocket and dRPC feel like they own the `byte[]` being sent between them. This can lead to data races, in which both `dRPC` and the websocket are writing. This is just tracking some experimentation to fix that race condition ## Run results: ## - Run 1: peer test failure - Run 2: peer test failure - Run 3: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040858460?check_suite_focus=true#step:8:45 ``` status code 412: The provided project history is running. Wait for it to complete importing!` ``` - Run 4: `TestWorkspaceHistory/CreateHistory` - https://github.com/coder/coder/runs/5040957999?check_suite_focus=true#step:7:176 ``` workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` - Run 5: peer failure - Run 6: Pass ✅ - Run 7: Peer failure ## Open Questions: ## ### Is `dRPC` or `websocket` at fault for the data race? It looks like this condition is specifically happening when `dRPC` decides to [`SendError`]). This constructs a new byte payload from [`MarshalError`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/error.go#L15) - so `dRPC` has created this buffer and owns it. From `dRPC`'s perspective, the callstack looks like this: - [`sendPacket`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcstream/stream.go#L253) - [`writeFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L65) - [`AppendFrame`](https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/packet.go#L128) - with finally the data race happening here: ```go // AppendFrame appends a marshaled form of the frame to the provided buffer. func AppendFrame(buf []byte, fr Frame) []byte { ... out := buf out = append(out, control). // <--------- ``` This should be fine, since `dPRC` create this buffer, and is taking the byte buffer constructed from `MarshalError` and tacking a bunch of headers on it to create a proper frame. Once `dRPC` is done writing, it _hangs onto the buffer and resets it here__: https://github.com/storj/drpc/blob/f6e369438f636b47ee788095d3fc13062ffbd019/drpcwire/writer.go#L73 However... the websocket implementation, once it gets the buffer, it runs a `statelessDeflate` [here](https://github.com/nhooyr/websocket/blob/8dee580a7f74cf1713400307b4eee514b927870f/write.go#L180), which compresses the buffer on the fly. This functionality actually [mutates the buffer in place](https://github.com/klauspost/compress/blob/a1a9cfc821f00faf2f5231beaa96244344d50391/flate/stateless.go#L94), which is where get our race. In the case where the `byte[]` aren't being manipulated anywhere else, this compress-in-place operation would be safe, and that's probably the case for most over-the-wire usages. In this case, though, where we're plumbing `dRPC` -> websocket, they both are manipulating it (`dRPC` is reusing the buffer for the next `write`, and `websocket` is compressing on the fly). ### Why does cloning on `Read` fail? Get a bunch of errors like: ``` 2022/02/02 19:26:10 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [ERR] yamux: Failed to read header: unexpected EOF 2022/02/02 19:26:25 [WARN] yamux: frame for missing stream: Vsn:0 Type:0 Flags:0 StreamID:0 Length:0 ``` # UPDATE: We decided we could disable websocket compression, which would avoid the race because the in-place `deflate` operaton would no longer be run. Trying that out now: - Run 1: ✅ - Run 2: https://github.com/coder/coder/runs/5042645522?check_suite_focus=true#step:8:338 - Run 3: ✅ - Run 4: https://github.com/coder/coder/runs/5042988758?check_suite_focus=true#step:7:168 - Run 5: ✅ * fix: Remove race condition with acquiredJobDone channel (#148) Found another data race while running the tests: https://github.com/coder/coder/runs/5044320845?check_suite_focus=true#step:7:83 __Issue:__ There is a race in the p.acquiredJobDone chan - in particular, there can be a case where we're waiting on the channel to finish (in close) with <-p.acquiredJobDone, but in parallel, an acquireJob could've been started, which would create a new channel for p.acquiredJobDone. There is a similar race in `close(..)`ing the channel, which also came up in test runs. __Fix:__ Instead of recreating the channel everytime, we can use `sync.WaitGroup` to accomplish the same functionality - a semaphore to make close wait for the current job to wrap up. * fix: Bump up workspace history timeout (#149) This is an attempted fix for failures like: https://github.com/coder/coder/runs/5043435263?check_suite_focus=true#step:7:32 Looking at the timing of the test: ``` t.go:56: 2022-02-02 21:33:21.964 [DEBUG] (terraform-provisioner) <provision.go:139> ran apply t.go:56: 2022-02-02 21:33:21.991 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.050 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.090 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.140 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.195 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running t.go:56: 2022-02-02 21:33:22.240 [DEBUG] (provisionerd) <provisionerd.go:162> skipping acquire; job is already running workspacehistory_test.go:122: Error Trace: workspacehistory_test.go:122 Error: Condition never satisfied Test: TestWorkspaceHistory/CreateHistory ``` It appears that the `terraform apply` job had just finished - with less than a second to spare until our `require.Eventually` completes - but there's still work to be done (ie, collecting the state files). So my suspicion is that terraform might, in some cases, exceed our 5s timeout. Note that in the setup for this test - there is a similar project history wait that waits for 15s, so I borrowed that here. In the future - we can look at potentially using a simple echo provider to exercise this in the unit test, in a way that is more reliable in terms of timing. I'll log an issue to track that. Co-authored-by: Bryan <bryan@coder.com>
2022-02-03 20:34:50 +00:00
require.NoError(t, closer.Close())
})
t.Run("TemplateImport", func(t *testing.T) {
t.Parallel()
var (
didComplete atomic.Bool
didLog atomic.Bool
didAcquireJob atomic.Bool
didDryRun = atomic.NewBool(true)
didReadme atomic.Bool
completeChan = make(chan struct{})
completeOnce sync.Once
)
closer := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
if !didAcquireJob.CAS(false, true) {
completeOnce.Do(func() { close(completeChan) })
return &proto.AcquiredJob{}, nil
}
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
"test.txt": "content",
runner.ReadmeFile: "# A cool template 😎\n",
}),
Type: &proto.AcquiredJob_TemplateImport_{
TemplateImport: &proto.AcquiredJob_TemplateImport{
Metadata: &sdkproto.Provision_Metadata{},
},
},
}, nil
},
updateJob: func(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error) {
if len(update.Logs) > 0 {
didLog.Store(true)
}
if len(update.Readme) > 0 {
didReadme.Store(true)
}
return &proto.UpdateJobResponse{}, nil
},
completeJob: func(ctx context.Context, job *proto.CompletedJob) (*proto.Empty, error) {
didComplete.Store(true)
return &proto.Empty{}, nil
},
}), nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{
parse: func(request *sdkproto.Parse_Request, stream sdkproto.DRPCProvisioner_ParseStream) error {
data, err := os.ReadFile(filepath.Join(request.Directory, "test.txt"))
require.NoError(t, err)
require.Equal(t, "content", string(data))
err = stream.Send(&sdkproto.Parse_Response{
Type: &sdkproto.Parse_Response_Log{
Log: &sdkproto.Log{
Level: sdkproto.LogLevel_INFO,
Output: "hello",
},
},
})
require.NoError(t, err)
err = stream.Send(&sdkproto.Parse_Response{
Type: &sdkproto.Parse_Response_Complete{
Complete: &sdkproto.Parse_Complete{},
},
})
require.NoError(t, err)
return nil
},
provision: func(stream sdkproto.DRPCProvisioner_ProvisionStream) error {
request, err := stream.Recv()
require.NoError(t, err)
if request.GetApply() != nil {
didDryRun.Store(false)
}
err = stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Log{
Log: &sdkproto.Log{
Level: sdkproto.LogLevel_INFO,
Output: "hello",
},
},
})
require.NoError(t, err)
err = stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Complete{
Complete: &sdkproto.Provision_Complete{
Resources: []*sdkproto.Resource{},
},
},
})
require.NoError(t, err)
return nil
},
}),
})
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, closer.Close())
assert.True(t, didLog.Load(), "should log some updates")
assert.True(t, didComplete.Load(), "should complete the job")
assert.True(t, didDryRun.Load(), "should be a dry run")
})
t.Run("TemplateDryRun", func(t *testing.T) {
t.Parallel()
var (
didComplete atomic.Bool
didLog atomic.Bool
didAcquireJob atomic.Bool
completeChan = make(chan struct{})
completeOnce sync.Once
parameterValues = []*sdkproto.ParameterValue{
{
DestinationScheme: sdkproto.ParameterDestination_PROVISIONER_VARIABLE,
Name: "test_var",
Value: "dean was here",
},
{
DestinationScheme: sdkproto.ParameterDestination_PROVISIONER_VARIABLE,
Name: "test_var_2",
Value: "1234",
},
}
metadata = &sdkproto.Provision_Metadata{}
)
closer := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
if !didAcquireJob.CAS(false, true) {
completeOnce.Do(func() { close(completeChan) })
return &proto.AcquiredJob{}, nil
}
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
"test.txt": "content",
}),
Type: &proto.AcquiredJob_TemplateDryRun_{
TemplateDryRun: &proto.AcquiredJob_TemplateDryRun{
ParameterValues: parameterValues,
Metadata: metadata,
},
},
}, nil
},
updateJob: func(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error) {
if len(update.Logs) == 0 {
t.Log("provisionerDaemonTestServer: no log messages")
return &proto.UpdateJobResponse{}, nil
}
didLog.Store(true)
for _, msg := range update.Logs {
t.Log("provisionerDaemonTestServer", "msg:", msg)
}
return &proto.UpdateJobResponse{}, nil
},
completeJob: func(ctx context.Context, job *proto.CompletedJob) (*proto.Empty, error) {
didComplete.Store(true)
return &proto.Empty{}, nil
},
}), nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{
provision: func(stream sdkproto.DRPCProvisioner_ProvisionStream) error {
err := stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Complete{
Complete: &sdkproto.Provision_Complete{
Resources: []*sdkproto.Resource{},
},
},
})
require.NoError(t, err)
return nil
},
}),
})
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, closer.Close())
assert.True(t, didLog.Load(), "should log some updates")
assert.True(t, didComplete.Load(), "should complete the job")
})
t.Run("WorkspaceBuild", func(t *testing.T) {
t.Parallel()
var (
didComplete atomic.Bool
didLog atomic.Bool
didAcquireJob atomic.Bool
completeChan = make(chan struct{})
completeOnce sync.Once
)
closer := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
if !didAcquireJob.CAS(false, true) {
completeOnce.Do(func() { close(completeChan) })
return &proto.AcquiredJob{}, nil
}
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
"test.txt": "content",
}),
Type: &proto.AcquiredJob_WorkspaceBuild_{
WorkspaceBuild: &proto.AcquiredJob_WorkspaceBuild{
Metadata: &sdkproto.Provision_Metadata{},
},
},
}, nil
},
updateJob: func(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error) {
if len(update.Logs) != 0 {
didLog.Store(true)
}
return &proto.UpdateJobResponse{}, nil
},
completeJob: func(ctx context.Context, job *proto.CompletedJob) (*proto.Empty, error) {
didComplete.Store(true)
return &proto.Empty{}, nil
},
}), nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{
provision: func(stream sdkproto.DRPCProvisioner_ProvisionStream) error {
err := stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Log{
Log: &sdkproto.Log{
Level: sdkproto.LogLevel_DEBUG,
Output: "wow",
},
},
})
require.NoError(t, err)
err = stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Complete{
Complete: &sdkproto.Provision_Complete{},
},
})
require.NoError(t, err)
return nil
},
}),
})
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, closer.Close())
assert.True(t, didLog.Load(), "should log some updates")
assert.True(t, didComplete.Load(), "should complete the job")
})
t.Run("WorkspaceBuildQuotaExceeded", func(t *testing.T) {
t.Parallel()
var (
didComplete atomic.Bool
didLog atomic.Bool
didAcquireJob atomic.Bool
didFail atomic.Bool
completeChan = make(chan struct{})
completeOnce sync.Once
)
closer := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
if !didAcquireJob.CAS(false, true) {
completeOnce.Do(func() { close(completeChan) })
return &proto.AcquiredJob{}, nil
}
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
"test.txt": "content",
}),
Type: &proto.AcquiredJob_WorkspaceBuild_{
WorkspaceBuild: &proto.AcquiredJob_WorkspaceBuild{
Metadata: &sdkproto.Provision_Metadata{},
},
},
}, nil
},
updateJob: func(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error) {
if len(update.Logs) != 0 {
didLog.Store(true)
}
return &proto.UpdateJobResponse{}, nil
},
completeJob: func(ctx context.Context, job *proto.CompletedJob) (*proto.Empty, error) {
didComplete.Store(true)
return &proto.Empty{}, nil
},
commitQuota: func(ctx context.Context, com *proto.CommitQuotaRequest) (*proto.CommitQuotaResponse, error) {
return &proto.CommitQuotaResponse{
Ok: com.DailyCost < 20,
}, nil
},
failJob: func(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error) {
didFail.Store(true)
return &proto.Empty{}, nil
},
}), nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{
provision: func(stream sdkproto.DRPCProvisioner_ProvisionStream) error {
err := stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Log{
Log: &sdkproto.Log{
Level: sdkproto.LogLevel_DEBUG,
Output: "wow",
},
},
})
require.NoError(t, err)
err = stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Complete{
Complete: &sdkproto.Provision_Complete{
Resources: []*sdkproto.Resource{
{
DailyCost: 10,
},
{
DailyCost: 15,
},
},
},
},
})
require.NoError(t, err)
return nil
},
}),
})
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, closer.Close())
assert.True(t, didLog.Load(), "should log some updates")
assert.False(t, didComplete.Load(), "should complete the job")
assert.True(t, didFail.Load(), "should fail the job")
})
t.Run("WorkspaceBuildFailComplete", func(t *testing.T) {
t.Parallel()
var (
didFail atomic.Bool
didAcquireJob atomic.Bool
completeChan = make(chan struct{})
completeOnce sync.Once
)
closer := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
if !didAcquireJob.CAS(false, true) {
completeOnce.Do(func() { close(completeChan) })
return &proto.AcquiredJob{}, nil
}
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
"test.txt": "content",
}),
Type: &proto.AcquiredJob_WorkspaceBuild_{
WorkspaceBuild: &proto.AcquiredJob_WorkspaceBuild{
Metadata: &sdkproto.Provision_Metadata{},
},
},
}, nil
},
updateJob: noopUpdateJob,
failJob: func(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error) {
didFail.Store(true)
return &proto.Empty{}, nil
},
}), nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{
provision: func(stream sdkproto.DRPCProvisioner_ProvisionStream) error {
return stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Complete{
Complete: &sdkproto.Provision_Complete{
Error: "some error",
},
},
})
},
}),
})
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, closer.Close())
assert.True(t, didFail.Load(), "should fail the job")
})
t.Run("Shutdown", func(t *testing.T) {
t.Parallel()
var updated sync.Once
var completed sync.Once
updateChan := make(chan struct{})
completeChan := make(chan struct{})
server := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
"test.txt": "content",
}),
Type: &proto.AcquiredJob_WorkspaceBuild_{
WorkspaceBuild: &proto.AcquiredJob_WorkspaceBuild{
Metadata: &sdkproto.Provision_Metadata{},
},
},
}, nil
},
updateJob: func(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error) {
if len(update.Logs) > 0 {
for _, log := range update.Logs {
if log.Source != proto.LogSource_PROVISIONER {
continue
}
// Close on a log so we know when the job is in progress!
updated.Do(func() {
close(updateChan)
})
break
}
}
return &proto.UpdateJobResponse{}, nil
},
failJob: func(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error) {
completed.Do(func() {
close(completeChan)
})
return &proto.Empty{}, nil
},
}), nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{
provision: func(stream sdkproto.DRPCProvisioner_ProvisionStream) error {
// Ignore the first provision message!
_, _ = stream.Recv()
err := stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Log{
Log: &sdkproto.Log{
Level: sdkproto.LogLevel_DEBUG,
Output: "in progress",
},
},
})
require.NoError(t, err)
msg, err := stream.Recv()
require.NoError(t, err)
require.NotNil(t, msg.GetCancel())
return stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Complete{
Complete: &sdkproto.Provision_Complete{
Error: "some error",
},
},
})
},
}),
})
require.Condition(t, closedWithin(updateChan, testutil.WaitShort))
err := server.Shutdown(context.Background())
require.NoError(t, err)
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, server.Close())
})
t.Run("ShutdownFromJob", func(t *testing.T) {
t.Parallel()
var completed sync.Once
var updated sync.Once
updateChan := make(chan struct{})
completeChan := make(chan struct{})
server := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
"test.txt": "content",
}),
Type: &proto.AcquiredJob_WorkspaceBuild_{
WorkspaceBuild: &proto.AcquiredJob_WorkspaceBuild{
Metadata: &sdkproto.Provision_Metadata{},
},
},
}, nil
},
updateJob: func(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error) {
resp := &proto.UpdateJobResponse{}
if len(update.Logs) > 0 {
for _, log := range update.Logs {
if log.Source != proto.LogSource_PROVISIONER {
continue
}
// Close on a log so we know when the job is in progress!
updated.Do(func() {
close(updateChan)
})
break
}
}
// start returning Canceled once we've gotten at least one log.
select {
case <-updateChan:
resp.Canceled = true
default:
// pass
}
return resp, nil
},
failJob: func(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error) {
completed.Do(func() {
close(completeChan)
})
return &proto.Empty{}, nil
},
}), nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{
provision: func(stream sdkproto.DRPCProvisioner_ProvisionStream) error {
// Ignore the first provision message!
_, _ = stream.Recv()
err := stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Log{
Log: &sdkproto.Log{
Level: sdkproto.LogLevel_DEBUG,
Output: "in progress",
},
},
})
require.NoError(t, err)
msg, err := stream.Recv()
require.NoError(t, err)
require.NotNil(t, msg.GetCancel())
return stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Complete{
Complete: &sdkproto.Provision_Complete{
Error: "some error",
},
},
})
},
}),
})
require.Condition(t, closedWithin(updateChan, testutil.WaitShort))
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, server.Close())
})
t.Run("ReconnectAndFail", func(t *testing.T) {
t.Parallel()
var (
second atomic.Bool
failChan = make(chan struct{})
failOnce sync.Once
failedChan = make(chan struct{})
failedOnce sync.Once
completeChan = make(chan struct{})
completeOnce sync.Once
)
server := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
client := createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
if second.Load() {
return &proto.AcquiredJob{}, nil
}
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
"test.txt": "content",
}),
Type: &proto.AcquiredJob_WorkspaceBuild_{
WorkspaceBuild: &proto.AcquiredJob_WorkspaceBuild{
Metadata: &sdkproto.Provision_Metadata{},
},
},
}, nil
},
updateJob: func(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error) {
return &proto.UpdateJobResponse{}, nil
},
failJob: func(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error) {
assert.Equal(t, job.JobId, "test")
if second.Load() {
completeOnce.Do(func() { close(completeChan) })
return &proto.Empty{}, nil
}
failOnce.Do(func() { close(failChan) })
<-failedChan
return &proto.Empty{}, nil
},
})
if !second.Load() {
go func() {
<-failChan
_ = client.DRPCConn().Close()
second.Store(true)
time.Sleep(50 * time.Millisecond)
failedOnce.Do(func() { close(failedChan) })
}()
}
return client, nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{
provision: func(stream sdkproto.DRPCProvisioner_ProvisionStream) error {
// Ignore the first provision message!
_, _ = stream.Recv()
return stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Complete{
Complete: &sdkproto.Provision_Complete{
Error: "some error",
},
},
})
},
}),
})
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, server.Close())
})
t.Run("ReconnectAndComplete", func(t *testing.T) {
t.Parallel()
var (
second atomic.Bool
failChan = make(chan struct{})
failOnce sync.Once
failedChan = make(chan struct{})
failedOnce sync.Once
completeChan = make(chan struct{})
completeOnce sync.Once
)
server := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
client := createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
if second.Load() {
completeOnce.Do(func() { close(completeChan) })
return &proto.AcquiredJob{}, nil
}
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
"test.txt": "content",
}),
Type: &proto.AcquiredJob_WorkspaceBuild_{
WorkspaceBuild: &proto.AcquiredJob_WorkspaceBuild{
Metadata: &sdkproto.Provision_Metadata{},
},
},
}, nil
},
failJob: func(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error) {
return nil, yamux.ErrSessionShutdown
},
updateJob: func(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error) {
return &proto.UpdateJobResponse{}, nil
},
completeJob: func(ctx context.Context, job *proto.CompletedJob) (*proto.Empty, error) {
if second.Load() {
return &proto.Empty{}, nil
}
failOnce.Do(func() { close(failChan) })
<-failedChan
return &proto.Empty{}, nil
},
})
if !second.Load() {
go func() {
<-failChan
_ = client.DRPCConn().Close()
second.Store(true)
time.Sleep(50 * time.Millisecond)
failedOnce.Do(func() { close(failedChan) })
}()
}
return client, nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{
provision: func(stream sdkproto.DRPCProvisioner_ProvisionStream) error {
// Ignore the first provision message!
_, _ = stream.Recv()
return stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Complete{
Complete: &sdkproto.Provision_Complete{},
},
})
},
}),
})
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, server.Close())
})
t.Run("UpdatesBeforeComplete", func(t *testing.T) {
t.Parallel()
logger := slogtest.Make(t, nil)
m := sync.Mutex{}
var ops []string
completeChan := make(chan struct{})
completeOnce := sync.Once{}
server := createProvisionerd(t, func(ctx context.Context) (proto.DRPCProvisionerDaemonClient, error) {
return createProvisionerDaemonClient(t, provisionerDaemonTestServer{
acquireJob: func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error) {
m.Lock()
defer m.Unlock()
logger.Info(ctx, "AcquiredJob called.")
if len(ops) > 0 {
return &proto.AcquiredJob{}, nil
}
ops = append(ops, "AcquireJob")
return &proto.AcquiredJob{
JobId: "test",
Provisioner: "someprovisioner",
TemplateSourceArchive: createTar(t, map[string]string{
"test.txt": "content",
}),
Type: &proto.AcquiredJob_WorkspaceBuild_{
WorkspaceBuild: &proto.AcquiredJob_WorkspaceBuild{
Metadata: &sdkproto.Provision_Metadata{},
},
},
}, nil
},
updateJob: func(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error) {
m.Lock()
defer m.Unlock()
logger.Info(ctx, "UpdateJob called.")
ops = append(ops, "UpdateJob")
for _, log := range update.Logs {
ops = append(ops, fmt.Sprintf("Log: %s | %s", log.Stage, log.Output))
}
return &proto.UpdateJobResponse{}, nil
},
completeJob: func(ctx context.Context, job *proto.CompletedJob) (*proto.Empty, error) {
m.Lock()
defer m.Unlock()
logger.Info(ctx, "CompleteJob called.")
ops = append(ops, "CompleteJob")
completeOnce.Do(func() { close(completeChan) })
return &proto.Empty{}, nil
},
failJob: func(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error) {
m.Lock()
defer m.Unlock()
logger.Info(ctx, "FailJob called.")
ops = append(ops, "FailJob")
return &proto.Empty{}, nil
},
}), nil
}, provisionerd.Provisioners{
"someprovisioner": createProvisionerClient(t, provisionerTestServer{
provision: func(stream sdkproto.DRPCProvisioner_ProvisionStream) error {
err := stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Log{
Log: &sdkproto.Log{
Level: sdkproto.LogLevel_DEBUG,
Output: "wow",
},
},
})
require.NoError(t, err)
err = stream.Send(&sdkproto.Provision_Response{
Type: &sdkproto.Provision_Response_Complete{
Complete: &sdkproto.Provision_Complete{},
},
})
require.NoError(t, err)
return nil
},
}),
})
require.Condition(t, closedWithin(completeChan, testutil.WaitShort))
require.NoError(t, server.Close())
assert.Equal(t, ops[len(ops)-1], "CompleteJob")
assert.Contains(t, ops[0:len(ops)-1], "Log: Cleaning Up | ")
})
}
// Creates an in-memory tar of the files provided.
func createTar(t *testing.T, files map[string]string) []byte {
var buffer bytes.Buffer
writer := tar.NewWriter(&buffer)
for path, content := range files {
err := writer.WriteHeader(&tar.Header{
Name: path,
Size: int64(len(content)),
})
require.NoError(t, err)
_, err = writer.Write([]byte(content))
require.NoError(t, err)
}
err := writer.Flush()
require.NoError(t, err)
return buffer.Bytes()
}
// Creates a provisionerd implementation with the provided dialer and provisioners.
func createProvisionerd(t *testing.T, dialer provisionerd.Dialer, provisioners provisionerd.Provisioners) *provisionerd.Server {
server := provisionerd.New(dialer, &provisionerd.Options{
Logger: slogtest.Make(t, nil).Named("provisionerd").Leveled(slog.LevelDebug),
JobPollInterval: 50 * time.Millisecond,
UpdateInterval: 50 * time.Millisecond,
Provisioners: provisioners,
WorkDirectory: t.TempDir(),
})
t.Cleanup(func() {
_ = server.Close()
})
return server
}
// Creates a provisionerd protobuf client that's connected
// to the server implementation provided.
func createProvisionerDaemonClient(t *testing.T, server provisionerDaemonTestServer) proto.DRPCProvisionerDaemonClient {
t.Helper()
if server.failJob == nil {
// Default to asserting the error from the failure, otherwise
// it can be lost in tests!
server.failJob = func(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error) {
assert.Fail(t, job.Error)
return &proto.Empty{}, nil
}
}
clientPipe, serverPipe := provisionersdk.MemTransportPipe()
t.Cleanup(func() {
_ = clientPipe.Close()
_ = serverPipe.Close()
})
mux := drpcmux.New()
err := proto.DRPCRegisterProvisionerDaemon(mux, &server)
require.NoError(t, err)
srv := drpcserver.New(mux)
ctx, cancelFunc := context.WithCancel(context.Background())
t.Cleanup(cancelFunc)
go func() {
_ = srv.Serve(ctx, serverPipe)
}()
return proto.NewDRPCProvisionerDaemonClient(clientPipe)
}
// Creates a provisioner protobuf client that's connected
// to the server implementation provided.
func createProvisionerClient(t *testing.T, server provisionerTestServer) sdkproto.DRPCProvisionerClient {
t.Helper()
clientPipe, serverPipe := provisionersdk.MemTransportPipe()
t.Cleanup(func() {
_ = clientPipe.Close()
_ = serverPipe.Close()
})
mux := drpcmux.New()
err := sdkproto.DRPCRegisterProvisioner(mux, &server)
require.NoError(t, err)
srv := drpcserver.New(mux)
ctx, cancelFunc := context.WithCancel(context.Background())
t.Cleanup(cancelFunc)
go func() {
_ = srv.Serve(ctx, serverPipe)
}()
return sdkproto.NewDRPCProvisionerClient(clientPipe)
}
type provisionerTestServer struct {
parse func(request *sdkproto.Parse_Request, stream sdkproto.DRPCProvisioner_ParseStream) error
provision func(stream sdkproto.DRPCProvisioner_ProvisionStream) error
}
func (p *provisionerTestServer) Parse(request *sdkproto.Parse_Request, stream sdkproto.DRPCProvisioner_ParseStream) error {
return p.parse(request, stream)
}
func (p *provisionerTestServer) Provision(stream sdkproto.DRPCProvisioner_ProvisionStream) error {
return p.provision(stream)
}
// Fulfills the protobuf interface for a ProvisionerDaemon with
// passable functions for dynamic functionality.
type provisionerDaemonTestServer struct {
acquireJob func(ctx context.Context, _ *proto.Empty) (*proto.AcquiredJob, error)
commitQuota func(ctx context.Context, com *proto.CommitQuotaRequest) (*proto.CommitQuotaResponse, error)
updateJob func(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error)
failJob func(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error)
completeJob func(ctx context.Context, job *proto.CompletedJob) (*proto.Empty, error)
}
func (p *provisionerDaemonTestServer) AcquireJob(ctx context.Context, empty *proto.Empty) (*proto.AcquiredJob, error) {
return p.acquireJob(ctx, empty)
}
func (p *provisionerDaemonTestServer) CommitQuota(ctx context.Context, com *proto.CommitQuotaRequest) (*proto.CommitQuotaResponse, error) {
if p.commitQuota == nil {
return &proto.CommitQuotaResponse{
Ok: true,
}, nil
}
return p.commitQuota(ctx, com)
}
func (p *provisionerDaemonTestServer) UpdateJob(ctx context.Context, update *proto.UpdateJobRequest) (*proto.UpdateJobResponse, error) {
return p.updateJob(ctx, update)
}
func (p *provisionerDaemonTestServer) FailJob(ctx context.Context, job *proto.FailedJob) (*proto.Empty, error) {
return p.failJob(ctx, job)
}
func (p *provisionerDaemonTestServer) CompleteJob(ctx context.Context, job *proto.CompletedJob) (*proto.Empty, error) {
return p.completeJob(ctx, job)
}