mirror of
https://github.com/actions/runner.git
synced 2025-12-10 20:36:49 +00:00
Compare commits
1 Commits
DontAddTem
...
v2.303.0
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
e676c78718 |
@@ -1,65 +0,0 @@
|
||||
# ADR 2494: Runner Image Tags
|
||||
|
||||
**Date**: 2023-03-17
|
||||
|
||||
**Status**: Accepted<!-- |Accepted|Rejected|Superceded|Deprecated -->
|
||||
|
||||
## Context
|
||||
|
||||
Following the [adoption of actions-runner-controller by GitHub](https://github.com/actions/actions-runner-controller/discussions/2072) and the introduction of the new runner scale set autoscaling mode, we needed to provide a basic runner image that could be used off the shelf without much friction.
|
||||
|
||||
The [current runner image](https://github.com/actions/runner/pkgs/container/actions-runner) is published to GHCR. Each release of this image is tagged with the runner version and the most recent release is also tagged with `latest`.
|
||||
|
||||
While the use of `latest` is common practice, we recommend that users pin a specific version of the runner image for a predictable runtime and improved security posture. However, we still notice that a large number of end users are relying on the `latest` tag & raising issues when they encounter problems.
|
||||
|
||||
Add to that, the community actions-runner-controller maintainers have issued a [deprecation notice](https://github.com/actions/actions-runner-controller/issues/2056) of the `latest` tag for the existing runner images (https://github.com/orgs/actions-runner-controller/packages).
|
||||
|
||||
## Decision
|
||||
|
||||
Proceed with Option 2, keeping the `latest` tag and adding the `NOTES.txt` file to our helm charts with the notice.
|
||||
|
||||
### Option 1: Remove the `latest` tag
|
||||
|
||||
By removing the `latest` tag, we have to proceed with either of these options:
|
||||
|
||||
1. Remove the runner image reference in the `values.yaml` provided with the `gha-runner-scale-set` helm chart and mark these fields as required so that users have to explicitly specify a runner image and a specific tag. This will obviously introduce more friction for users who want to start using actions-runner-controller for the first time.
|
||||
|
||||
```yaml
|
||||
spec:
|
||||
containers:
|
||||
- name: runner
|
||||
image: ""
|
||||
tag: ""
|
||||
command: ["/home/runner/run.sh"]
|
||||
```
|
||||
|
||||
1. Pin a specific runner image tag in the `values.yaml` provided with the `gha-runner-scale-set` helm chart. This will reduce friction for users who want to start using actions-runner-controller for the first time but will require us to update the `values.yaml` with every new runner release.
|
||||
|
||||
```yaml
|
||||
spec:
|
||||
containers:
|
||||
- name: runner
|
||||
image: "ghcr.io/actions/actions-runner"
|
||||
tag: "v2.300.0"
|
||||
command: ["/home/runner/run.sh"]
|
||||
```
|
||||
|
||||
### Option 2: Keep the `latest` tag
|
||||
|
||||
Keeping the `latest` tag is also a reasonable option especially if we don't expect to make any breaking changes to the runner image. We could enhance this by adding a [NOTES.txt](https://helm.sh/docs/chart_template_guide/notes_files/) to the helm chart which will be displayed to the user after a successful helm install/upgrade. This will help users understand the implications of using the `latest` tag and how to pin a specific version of the runner image.
|
||||
|
||||
The runner image release workflow will need to be updated so that the image is pushed to GHCR and tagged only when the runner rollout has reached all scale units.
|
||||
|
||||
## Consequences
|
||||
|
||||
Proceeding with **option 1** means:
|
||||
|
||||
1. We will enhance the runtime predictability and security posture of our end users
|
||||
1. We will have to update the `values.yaml` with every new runner release (that can be automated)
|
||||
1. We will introduce friction for users who want to start using actions-runner-controller for the first time
|
||||
|
||||
Proceeding with **option 2** means:
|
||||
|
||||
1. We will have to continue to maintain the `latest` tag
|
||||
1. We will assume that end users will be able to handle the implications of using the `latest` tag
|
||||
1. Runner image release workflow needs to be updated
|
||||
@@ -157,7 +157,7 @@ cat (Runner/Worker)_TIMESTAMP.log # view your log file
|
||||
## Styling
|
||||
|
||||
We use the .NET Foundation and CoreCLR style guidelines [located here](
|
||||
https://github.com/dotnet/runtime/blob/main/docs/coding-guidelines/coding-style.md)
|
||||
https://github.com/dotnet/corefx/blob/master/Documentation/coding-guidelines/coding-style.md)
|
||||
|
||||
### Format C# Code
|
||||
|
||||
|
||||
@@ -2,7 +2,7 @@ FROM mcr.microsoft.com/dotnet/runtime-deps:6.0 as build
|
||||
|
||||
ARG RUNNER_VERSION
|
||||
ARG RUNNER_ARCH="x64"
|
||||
ARG RUNNER_CONTAINER_HOOKS_VERSION=0.3.1
|
||||
ARG RUNNER_CONTAINER_HOOKS_VERSION=0.2.0
|
||||
ARG DOCKER_VERSION=20.10.23
|
||||
|
||||
RUN apt update -y && apt install curl unzip -y
|
||||
|
||||
@@ -1,18 +1,17 @@
|
||||
## Features
|
||||
- Runner changes for communication with Results service (#2510, #2531, #2535, #2516)
|
||||
- Add `*.ghe.localhost` domains to hosted server check (#2536)
|
||||
- Add `OrchestrationId` to user-agent for better telemetry correlation. (#2568)
|
||||
- Support matrix context in output keys (#2477)
|
||||
- Add update certificates to `./run.sh` if `RUNNER_UPDATE_CA_CERTS` env is set (#2471)
|
||||
- Bypass all proxies for all hosts if `no_proxy='*'` is set (#2395)
|
||||
- Change runner image to make user/folder align with `ubuntu-latest` hosted runner. (#2469)
|
||||
|
||||
## Bugs
|
||||
- Fix JIT configurations on Windows (#2497)
|
||||
- Guard against NullReference while creating HostContext (#2343)
|
||||
- Handles broken symlink in `Which` (#2150, #2196)
|
||||
- Adding curl retry for external tool downloads (#2552, #2557)
|
||||
- Limit the time we wait for waiting websocket to connect. (#2554)
|
||||
- Exit on runner version deprecation error (#2299)
|
||||
- Runner service exit after consecutive re-try exits (#2426)
|
||||
|
||||
## Misc
|
||||
- Bump container hooks version to 0.3.1 in runner image (#2496)
|
||||
- Runner changes to communicate with vNext services (#2487, #2500, #2505, #2541, #2547)
|
||||
- Replace deprecated command with environment file (#2429)
|
||||
- Make requests to `Run` service to renew job request (#2461)
|
||||
- Add job/step log upload to Result service (#2447, #2439)
|
||||
|
||||
_Note: Actions Runner follows a progressive release policy, so the latest release might not be available to your enterprise, organization, or repository yet.
|
||||
To confirm which version of the Actions Runner you should expect, please view the download instructions for your enterprise, organization, or repository.
|
||||
|
||||
@@ -1 +1 @@
|
||||
<Update to ./src/runnerversion when creating release>
|
||||
2.303.0
|
||||
|
||||
@@ -55,23 +55,12 @@ function acquireExternalTool() {
|
||||
# Download from source to the partial file.
|
||||
echo "Downloading $download_source"
|
||||
mkdir -p "$(dirname "$download_target")" || checkRC 'mkdir'
|
||||
|
||||
CURL_VERSION=$(curl --version | awk 'NR==1{print $2}')
|
||||
echo "Curl version: $CURL_VERSION"
|
||||
|
||||
# curl -f Fail silently (no output at all) on HTTP errors (H)
|
||||
# -k Allow connections to SSL sites without certs (H)
|
||||
# -S Show error. With -s, make curl show errors when they occur
|
||||
# -L Follow redirects (H)
|
||||
# -o FILE Write to FILE instead of stdout
|
||||
# --retry 3 Retries transient errors 3 times (timeouts, 5xx)
|
||||
if [[ "$(printf '%s\n' "7.71.0" "$CURL_VERSION" | sort -V | head -n1)" != "7.71.0" ]]; then
|
||||
# Curl version is less than or equal to 7.71.0, skipping retry-all-errors flag
|
||||
curl -fkSL --retry 3 -o "$partial_target" "$download_source" 2>"${download_target}_download.log" || checkRC 'curl'
|
||||
else
|
||||
# Curl version is greater than 7.71.0, running curl with --retry-all-errors flag
|
||||
curl -fkSL --retry 3 --retry-all-errors -o "$partial_target" "$download_source" 2>"${download_target}_download.log" || checkRC 'curl'
|
||||
fi
|
||||
curl -fkSL -o "$partial_target" "$download_source" 2>"${download_target}_download.log" || checkRC 'curl'
|
||||
|
||||
# Move the partial file to the download target.
|
||||
mv "$partial_target" "$download_target" || checkRC 'mv'
|
||||
|
||||
@@ -1,56 +0,0 @@
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
using GitHub.Actions.RunService.WebApi;
|
||||
using GitHub.DistributedTask.Pipelines;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using GitHub.Runner.Sdk;
|
||||
using GitHub.Services.Common;
|
||||
using Sdk.RSWebApi.Contracts;
|
||||
using Sdk.WebApi.WebApi.RawClient;
|
||||
|
||||
namespace GitHub.Runner.Common
|
||||
{
|
||||
[ServiceLocator(Default = typeof(BrokerServer))]
|
||||
public interface IBrokerServer : IRunnerService
|
||||
{
|
||||
Task ConnectAsync(Uri serverUrl, VssCredentials credentials);
|
||||
|
||||
Task<TaskAgentMessage> GetRunnerMessageAsync(CancellationToken token, TaskAgentStatus status, string version);
|
||||
}
|
||||
|
||||
public sealed class BrokerServer : RunnerService, IBrokerServer
|
||||
{
|
||||
private bool _hasConnection;
|
||||
private Uri _brokerUri;
|
||||
private RawConnection _connection;
|
||||
private BrokerHttpClient _brokerHttpClient;
|
||||
|
||||
public async Task ConnectAsync(Uri serverUri, VssCredentials credentials)
|
||||
{
|
||||
_brokerUri = serverUri;
|
||||
|
||||
_connection = VssUtil.CreateRawConnection(serverUri, credentials);
|
||||
_brokerHttpClient = await _connection.GetClientAsync<BrokerHttpClient>();
|
||||
_hasConnection = true;
|
||||
}
|
||||
|
||||
private void CheckConnection()
|
||||
{
|
||||
if (!_hasConnection)
|
||||
{
|
||||
throw new InvalidOperationException($"SetConnection");
|
||||
}
|
||||
}
|
||||
|
||||
public Task<TaskAgentMessage> GetRunnerMessageAsync(CancellationToken cancellationToken, TaskAgentStatus status, string version)
|
||||
{
|
||||
CheckConnection();
|
||||
var jobMessage = RetryRequest<TaskAgentMessage>(
|
||||
async () => await _brokerHttpClient.GetRunnerMessageAsync(version, status, cancellationToken), cancellationToken);
|
||||
|
||||
return jobMessage;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -50,12 +50,6 @@ namespace GitHub.Runner.Common
|
||||
[DataMember(EmitDefaultValue = false)]
|
||||
public string MonitorSocketAddress { get; set; }
|
||||
|
||||
[DataMember(EmitDefaultValue = false)]
|
||||
public bool UseV2Flow { get; set; }
|
||||
|
||||
[DataMember(EmitDefaultValue = false)]
|
||||
public string ServerUrlV2 { get; set; }
|
||||
|
||||
[IgnoreDataMember]
|
||||
public bool IsHostedServer
|
||||
{
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using System;
|
||||
using System;
|
||||
|
||||
namespace GitHub.Runner.Common
|
||||
{
|
||||
@@ -132,7 +132,6 @@ namespace GitHub.Runner.Common
|
||||
public static readonly string GenerateServiceConfig = "generateServiceConfig";
|
||||
public static readonly string Help = "help";
|
||||
public static readonly string Local = "local";
|
||||
public static readonly string NoDefaultLabels = "no-default-labels";
|
||||
public static readonly string Replace = "replace";
|
||||
public static readonly string DisableUpdate = "disableupdate";
|
||||
public static readonly string Once = "once"; // Keep this around since customers still relies on it
|
||||
@@ -262,8 +261,6 @@ namespace GitHub.Runner.Common
|
||||
public static readonly string AccessToken = "system.accessToken";
|
||||
public static readonly string Culture = "system.culture";
|
||||
public static readonly string PhaseDisplayName = "system.phaseDisplayName";
|
||||
public static readonly string JobRequestType = "system.jobRequestType";
|
||||
public static readonly string OrchestrationId = "system.orchestrationId";
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using System;
|
||||
using System;
|
||||
using System.Collections.Concurrent;
|
||||
using System.Collections.Generic;
|
||||
using System.Diagnostics;
|
||||
@@ -220,7 +220,7 @@ namespace GitHub.Runner.Common
|
||||
var runnerFile = GetConfigFile(WellKnownConfigFile.Runner);
|
||||
if (File.Exists(runnerFile))
|
||||
{
|
||||
var runnerSettings = IOUtil.LoadObject<RunnerSettings>(runnerFile, true);
|
||||
var runnerSettings = IOUtil.LoadObject<RunnerSettings>(runnerFile);
|
||||
_userAgents.Add(new ProductInfoHeaderValue("RunnerId", runnerSettings.AgentId.ToString(CultureInfo.InvariantCulture)));
|
||||
_userAgents.Add(new ProductInfoHeaderValue("GroupId", runnerSettings.PoolId.ToString(CultureInfo.InvariantCulture)));
|
||||
}
|
||||
|
||||
@@ -11,10 +11,10 @@ using System.Threading.Tasks;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using GitHub.Runner.Sdk;
|
||||
using GitHub.Services.Common;
|
||||
using GitHub.Services.OAuth;
|
||||
using GitHub.Services.Results.Client;
|
||||
using GitHub.Services.WebApi;
|
||||
using GitHub.Services.WebApi.Utilities.Internal;
|
||||
using GitHub.Services.Results.Client;
|
||||
using GitHub.Services.OAuth;
|
||||
|
||||
namespace GitHub.Runner.Common
|
||||
{
|
||||
@@ -24,11 +24,15 @@ namespace GitHub.Runner.Common
|
||||
Task ConnectAsync(VssConnection jobConnection);
|
||||
|
||||
void InitializeWebsocketClient(ServiceEndpoint serviceEndpoint);
|
||||
void InitializeResultsClient(Uri uri, string token);
|
||||
|
||||
// logging and console
|
||||
Task<TaskLog> AppendLogContentAsync(Guid scopeIdentifier, string hubName, Guid planId, int logId, Stream uploadStream, CancellationToken cancellationToken);
|
||||
Task AppendTimelineRecordFeedAsync(Guid scopeIdentifier, string hubName, Guid planId, Guid timelineId, Guid timelineRecordId, Guid stepId, IList<string> lines, long? startLine, CancellationToken cancellationToken);
|
||||
Task<TaskAttachment> CreateAttachmentAsync(Guid scopeIdentifier, string hubName, Guid planId, Guid timelineId, Guid timelineRecordId, String type, String name, Stream uploadStream, CancellationToken cancellationToken);
|
||||
Task CreateStepSummaryAsync(string planId, string jobId, Guid stepId, string file, CancellationToken cancellationToken);
|
||||
Task CreateResultsStepLogAsync(string planId, string jobId, Guid stepId, string file, bool finalize, bool firstBlock, long lineCount, CancellationToken cancellationToken);
|
||||
Task CreateResultsJobLogAsync(string planId, string jobId, string file, bool finalize, bool firstBlock, long lineCount, CancellationToken cancellationToken);
|
||||
Task<TaskLog> CreateLogAsync(Guid scopeIdentifier, string hubName, Guid planId, TaskLog log, CancellationToken cancellationToken);
|
||||
Task<Timeline> CreateTimelineAsync(Guid scopeIdentifier, string hubName, Guid planId, Guid timelineId, CancellationToken cancellationToken);
|
||||
Task<List<TimelineRecord>> UpdateTimelineRecordsAsync(Guid scopeIdentifier, string hubName, Guid planId, Guid timelineId, IEnumerable<TimelineRecord> records, CancellationToken cancellationToken);
|
||||
@@ -42,6 +46,7 @@ namespace GitHub.Runner.Common
|
||||
private bool _hasConnection;
|
||||
private VssConnection _connection;
|
||||
private TaskHttpClient _taskClient;
|
||||
private ResultsHttpClient _resultsClient;
|
||||
private ClientWebSocket _websocketClient;
|
||||
|
||||
private ServiceEndpoint _serviceEndpoint;
|
||||
@@ -145,6 +150,12 @@ namespace GitHub.Runner.Common
|
||||
InitializeWebsocketClient(TimeSpan.Zero);
|
||||
}
|
||||
|
||||
public void InitializeResultsClient(Uri uri, string token)
|
||||
{
|
||||
var httpMessageHandler = HostContext.CreateHttpClientHandler();
|
||||
this._resultsClient = new ResultsHttpClient(uri, httpMessageHandler, token, disposeHandler: true);
|
||||
}
|
||||
|
||||
public ValueTask DisposeAsync()
|
||||
{
|
||||
CloseWebSocket(WebSocketCloseStatus.NormalClosure, CancellationToken.None);
|
||||
@@ -199,15 +210,13 @@ namespace GitHub.Runner.Common
|
||||
{
|
||||
Trace.Info($"Attempting to start websocket client with delay {delay}.");
|
||||
await Task.Delay(delay);
|
||||
using var connectTimeoutTokenSource = new CancellationTokenSource(TimeSpan.FromSeconds(30));
|
||||
await this._websocketClient.ConnectAsync(new Uri(feedStreamUrl), connectTimeoutTokenSource.Token);
|
||||
await this._websocketClient.ConnectAsync(new Uri(feedStreamUrl), default(CancellationToken));
|
||||
Trace.Info($"Successfully started websocket client.");
|
||||
}
|
||||
catch (Exception ex)
|
||||
{
|
||||
Trace.Info("Exception caught during websocket client connect, fallback of HTTP would be used now instead of websocket.");
|
||||
Trace.Error(ex);
|
||||
this._websocketClient = null;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -254,7 +263,7 @@ namespace GitHub.Runner.Common
|
||||
{
|
||||
failedAttemptsToPostBatchedLinesByWebsocket++;
|
||||
Trace.Info($"Caught exception during append web console line to websocket, let's fallback to sending via non-websocket call (total calls: {totalBatchedLinesAttemptedByWebsocket}, failed calls: {failedAttemptsToPostBatchedLinesByWebsocket}, websocket state: {this._websocketClient?.State}).");
|
||||
Trace.Verbose(ex.ToString());
|
||||
Trace.Error(ex);
|
||||
if (totalBatchedLinesAttemptedByWebsocket > _minWebsocketBatchedLinesCountToConsider)
|
||||
{
|
||||
// let's consider failure percentage
|
||||
@@ -309,6 +318,32 @@ namespace GitHub.Runner.Common
|
||||
return _taskClient.CreateAttachmentAsync(scopeIdentifier, hubName, planId, timelineId, timelineRecordId, type, name, uploadStream, cancellationToken: cancellationToken);
|
||||
}
|
||||
|
||||
public Task CreateStepSummaryAsync(string planId, string jobId, Guid stepId, string file, CancellationToken cancellationToken)
|
||||
{
|
||||
if (_resultsClient != null)
|
||||
{
|
||||
return _resultsClient.UploadStepSummaryAsync(planId, jobId, stepId, file, cancellationToken: cancellationToken);
|
||||
}
|
||||
throw new InvalidOperationException("Results client is not initialized.");
|
||||
}
|
||||
|
||||
public Task CreateResultsStepLogAsync(string planId, string jobId, Guid stepId, string file, bool finalize, bool firstBlock, long lineCount, CancellationToken cancellationToken)
|
||||
{
|
||||
if (_resultsClient != null)
|
||||
{
|
||||
return _resultsClient.UploadResultsStepLogAsync(planId, jobId, stepId, file, finalize, firstBlock, lineCount, cancellationToken: cancellationToken);
|
||||
}
|
||||
throw new InvalidOperationException("Results client is not initialized.");
|
||||
}
|
||||
|
||||
public Task CreateResultsJobLogAsync(string planId, string jobId, string file, bool finalize, bool firstBlock, long lineCount, CancellationToken cancellationToken)
|
||||
{
|
||||
if (_resultsClient != null)
|
||||
{
|
||||
return _resultsClient.UploadResultsJobLogAsync(planId, jobId, file, finalize, firstBlock, lineCount, cancellationToken: cancellationToken);
|
||||
}
|
||||
throw new InvalidOperationException("Results client is not initialized.");
|
||||
}
|
||||
|
||||
public Task<TaskLog> CreateLogAsync(Guid scopeIdentifier, string hubName, Guid planId, TaskLog log, CancellationToken cancellationToken)
|
||||
{
|
||||
|
||||
@@ -17,7 +17,7 @@ namespace GitHub.Runner.Common
|
||||
TaskCompletionSource<int> JobRecordUpdated { get; }
|
||||
event EventHandler<ThrottlingEventArgs> JobServerQueueThrottling;
|
||||
Task ShutdownAsync();
|
||||
void Start(Pipelines.AgentJobRequestMessage jobRequest, bool resultServiceOnly = false);
|
||||
void Start(Pipelines.AgentJobRequestMessage jobRequest);
|
||||
void QueueWebConsoleLine(Guid stepRecordId, string line, long? lineNumber = null);
|
||||
void QueueFileUpload(Guid timelineId, Guid timelineRecordId, string type, string name, string path, bool deleteSource);
|
||||
void QueueResultsUpload(Guid timelineRecordId, string name, string path, string type, bool deleteSource, bool finalize, bool firstBlock, long totalLines);
|
||||
@@ -65,12 +65,10 @@ namespace GitHub.Runner.Common
|
||||
|
||||
// common
|
||||
private IJobServer _jobServer;
|
||||
private IResultsServer _resultsServer;
|
||||
private Task[] _allDequeueTasks;
|
||||
private readonly TaskCompletionSource<int> _jobCompletionSource = new();
|
||||
private readonly TaskCompletionSource<int> _jobRecordUpdated = new();
|
||||
private bool _queueInProcess = false;
|
||||
private bool _resultsServiceOnly = false;
|
||||
|
||||
public TaskCompletionSource<int> JobRecordUpdated => _jobRecordUpdated;
|
||||
|
||||
@@ -93,20 +91,15 @@ namespace GitHub.Runner.Common
|
||||
{
|
||||
base.Initialize(hostContext);
|
||||
_jobServer = hostContext.GetService<IJobServer>();
|
||||
_resultsServer = hostContext.GetService<IResultsServer>();
|
||||
}
|
||||
|
||||
public void Start(Pipelines.AgentJobRequestMessage jobRequest, bool resultServiceOnly = false)
|
||||
public void Start(Pipelines.AgentJobRequestMessage jobRequest)
|
||||
{
|
||||
Trace.Entering();
|
||||
_resultsServiceOnly = resultServiceOnly;
|
||||
|
||||
var serviceEndPoint = jobRequest.Resources.Endpoints.Single(x => string.Equals(x.Name, WellKnownServiceEndpointNames.SystemVssConnection, StringComparison.OrdinalIgnoreCase));
|
||||
|
||||
if (!resultServiceOnly)
|
||||
{
|
||||
_jobServer.InitializeWebsocketClient(serviceEndPoint);
|
||||
}
|
||||
_jobServer.InitializeWebsocketClient(serviceEndPoint);
|
||||
|
||||
// This code is usually wrapped by an instance of IExecutionContext which isn't available here.
|
||||
jobRequest.Variables.TryGetValue("system.github.results_endpoint", out VariableValue resultsEndpointVariable);
|
||||
@@ -117,16 +110,8 @@ namespace GitHub.Runner.Common
|
||||
!string.IsNullOrEmpty(accessToken) &&
|
||||
!string.IsNullOrEmpty(resultsReceiverEndpoint))
|
||||
{
|
||||
string liveConsoleFeedUrl = null;
|
||||
Trace.Info("Initializing results client");
|
||||
if (resultServiceOnly
|
||||
&& serviceEndPoint.Data.TryGetValue("FeedStreamUrl", out var feedStreamUrl)
|
||||
&& !string.IsNullOrEmpty(feedStreamUrl))
|
||||
{
|
||||
liveConsoleFeedUrl = feedStreamUrl;
|
||||
}
|
||||
|
||||
_resultsServer.InitializeResultsClient(new Uri(resultsReceiverEndpoint), liveConsoleFeedUrl, accessToken);
|
||||
_jobServer.InitializeResultsClient(new Uri(resultsReceiverEndpoint), accessToken);
|
||||
_resultsClientInitiated = true;
|
||||
}
|
||||
|
||||
@@ -207,9 +192,6 @@ namespace GitHub.Runner.Common
|
||||
Trace.Info($"Disposing job server ...");
|
||||
await _jobServer.DisposeAsync();
|
||||
|
||||
Trace.Info($"Disposing results server ...");
|
||||
await _resultsServer.DisposeAsync();
|
||||
|
||||
Trace.Info("All queue process tasks have been stopped, and all queues are drained.");
|
||||
}
|
||||
|
||||
@@ -388,14 +370,7 @@ namespace GitHub.Runner.Common
|
||||
// Give at most 60s for each request.
|
||||
using (var timeoutTokenSource = new CancellationTokenSource(TimeSpan.FromSeconds(60)))
|
||||
{
|
||||
if (_resultsServiceOnly)
|
||||
{
|
||||
await _resultsServer.AppendLiveConsoleFeedAsync(_scopeIdentifier, _hubName, _planId, _jobTimelineId, _jobTimelineRecordId, stepRecordId, batch.Select(logLine => logLine.Line).ToList(), batch[0].LineNumber, timeoutTokenSource.Token);
|
||||
}
|
||||
else
|
||||
{
|
||||
await _jobServer.AppendTimelineRecordFeedAsync(_scopeIdentifier, _hubName, _planId, _jobTimelineId, _jobTimelineRecordId, stepRecordId, batch.Select(logLine => logLine.Line).ToList(), batch[0].LineNumber, timeoutTokenSource.Token);
|
||||
}
|
||||
await _jobServer.AppendTimelineRecordFeedAsync(_scopeIdentifier, _hubName, _planId, _jobTimelineId, _jobTimelineRecordId, stepRecordId, batch.Select(logLine => logLine.Line).ToList(), batch[0].LineNumber, timeoutTokenSource.Token);
|
||||
}
|
||||
|
||||
if (_firstConsoleOutputs)
|
||||
@@ -537,14 +512,19 @@ namespace GitHub.Runner.Common
|
||||
}
|
||||
catch (Exception ex)
|
||||
{
|
||||
var issue = new Issue() { Type = IssueType.Warning, Message = $"Caught exception during file upload to results. {ex.Message}" };
|
||||
issue.Data[Constants.Runner.InternalTelemetryIssueDataKey] = Constants.Runner.ResultsUploadFailure;
|
||||
|
||||
var telemetryRecord = new TimelineRecord()
|
||||
{
|
||||
Id = Constants.Runner.TelemetryRecordId,
|
||||
};
|
||||
telemetryRecord.Issues.Add(issue);
|
||||
QueueTimelineRecordUpdate(_jobTimelineId, telemetryRecord);
|
||||
|
||||
Trace.Info("Catch exception during file upload to results, keep going since the process is best effort.");
|
||||
Trace.Error(ex);
|
||||
errorCount++;
|
||||
|
||||
// If we hit any exceptions uploading to Results, let's skip any additional uploads to Results
|
||||
_resultsClientInitiated = false;
|
||||
|
||||
SendResultsTelemetry(ex);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -562,19 +542,6 @@ namespace GitHub.Runner.Common
|
||||
}
|
||||
}
|
||||
|
||||
private void SendResultsTelemetry(Exception ex)
|
||||
{
|
||||
var issue = new Issue() { Type = IssueType.Warning, Message = $"Caught exception with results. {ex.Message}" };
|
||||
issue.Data[Constants.Runner.InternalTelemetryIssueDataKey] = Constants.Runner.ResultsUploadFailure;
|
||||
|
||||
var telemetryRecord = new TimelineRecord()
|
||||
{
|
||||
Id = Constants.Runner.TelemetryRecordId,
|
||||
};
|
||||
telemetryRecord.Issues.Add(issue);
|
||||
QueueTimelineRecordUpdate(_jobTimelineId, telemetryRecord);
|
||||
}
|
||||
|
||||
private async Task ProcessTimelinesUpdateQueueAsync(bool runOnce = false)
|
||||
{
|
||||
while (!_jobCompletionSource.Task.IsCompleted || runOnce)
|
||||
@@ -622,7 +589,7 @@ namespace GitHub.Runner.Common
|
||||
|
||||
foreach (var detailTimeline in update.PendingRecords.Where(r => r.Details != null))
|
||||
{
|
||||
if (!_resultsServiceOnly && !_allTimelines.Contains(detailTimeline.Details.Id))
|
||||
if (!_allTimelines.Contains(detailTimeline.Details.Id))
|
||||
{
|
||||
try
|
||||
{
|
||||
@@ -644,27 +611,7 @@ namespace GitHub.Runner.Common
|
||||
|
||||
try
|
||||
{
|
||||
if (!_resultsServiceOnly)
|
||||
{
|
||||
await _jobServer.UpdateTimelineRecordsAsync(_scopeIdentifier, _hubName, _planId, update.TimelineId, update.PendingRecords, default(CancellationToken));
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
if (_resultsClientInitiated)
|
||||
{
|
||||
await _resultsServer.UpdateResultsWorkflowStepsAsync(_scopeIdentifier, _hubName, _planId, update.TimelineId, update.PendingRecords, default(CancellationToken));
|
||||
}
|
||||
}
|
||||
catch (Exception e)
|
||||
{
|
||||
Trace.Info("Catch exception during update steps, skip update Results.");
|
||||
Trace.Error(e);
|
||||
_resultsClientInitiated = false;
|
||||
|
||||
SendResultsTelemetry(e);
|
||||
}
|
||||
|
||||
await _jobServer.UpdateTimelineRecordsAsync(_scopeIdentifier, _hubName, _planId, update.TimelineId, update.PendingRecords, default(CancellationToken));
|
||||
if (_bufferedRetryRecords.Remove(update.TimelineId))
|
||||
{
|
||||
Trace.Verbose("Cleanup buffered timeline record for timeline: {0}.", update.TimelineId);
|
||||
@@ -756,17 +703,17 @@ namespace GitHub.Runner.Common
|
||||
timelineRecord.State = rec.State ?? timelineRecord.State;
|
||||
timelineRecord.WorkerName = rec.WorkerName ?? timelineRecord.WorkerName;
|
||||
|
||||
if (rec.ErrorCount > 0)
|
||||
if (rec.ErrorCount != null && rec.ErrorCount > 0)
|
||||
{
|
||||
timelineRecord.ErrorCount = rec.ErrorCount;
|
||||
}
|
||||
|
||||
if (rec.WarningCount > 0)
|
||||
if (rec.WarningCount != null && rec.WarningCount > 0)
|
||||
{
|
||||
timelineRecord.WarningCount = rec.WarningCount;
|
||||
}
|
||||
|
||||
if (rec.NoticeCount > 0)
|
||||
if (rec.NoticeCount != null && rec.NoticeCount > 0)
|
||||
{
|
||||
timelineRecord.NoticeCount = rec.NoticeCount;
|
||||
}
|
||||
@@ -797,7 +744,7 @@ namespace GitHub.Runner.Common
|
||||
foreach (var record in mergedRecords)
|
||||
{
|
||||
Trace.Verbose($" Record: t={record.RecordType}, n={record.Name}, s={record.State}, st={record.StartTime}, {record.PercentComplete}%, ft={record.FinishTime}, r={record.Result}: {record.CurrentOperation}");
|
||||
if (record.Issues != null)
|
||||
if (record.Issues != null && record.Issues.Count > 0)
|
||||
{
|
||||
foreach (var issue in record.Issues)
|
||||
{
|
||||
@@ -807,7 +754,7 @@ namespace GitHub.Runner.Common
|
||||
}
|
||||
}
|
||||
|
||||
if (record.Variables != null)
|
||||
if (record.Variables != null && record.Variables.Count > 0)
|
||||
{
|
||||
foreach (var variable in record.Variables)
|
||||
{
|
||||
@@ -818,35 +765,33 @@ namespace GitHub.Runner.Common
|
||||
|
||||
return mergedRecords;
|
||||
}
|
||||
|
||||
private async Task UploadFile(UploadFileInfo file)
|
||||
{
|
||||
bool uploadSucceed = false;
|
||||
try
|
||||
{
|
||||
if (!_resultsServiceOnly)
|
||||
if (String.Equals(file.Type, CoreAttachmentType.Log, StringComparison.OrdinalIgnoreCase))
|
||||
{
|
||||
if (String.Equals(file.Type, CoreAttachmentType.Log, StringComparison.OrdinalIgnoreCase))
|
||||
// Create the log
|
||||
var taskLog = await _jobServer.CreateLogAsync(_scopeIdentifier, _hubName, _planId, new TaskLog(String.Format(@"logs\{0:D}", file.TimelineRecordId)), default(CancellationToken));
|
||||
|
||||
// Upload the contents
|
||||
using (FileStream fs = File.Open(file.Path, FileMode.Open, FileAccess.Read, FileShare.ReadWrite))
|
||||
{
|
||||
// Create the log
|
||||
var taskLog = await _jobServer.CreateLogAsync(_scopeIdentifier, _hubName, _planId, new TaskLog(String.Format(@"logs\{0:D}", file.TimelineRecordId)), default(CancellationToken));
|
||||
|
||||
// Upload the contents
|
||||
using (FileStream fs = File.Open(file.Path, FileMode.Open, FileAccess.Read, FileShare.ReadWrite))
|
||||
{
|
||||
var logUploaded = await _jobServer.AppendLogContentAsync(_scopeIdentifier, _hubName, _planId, taskLog.Id, fs, default(CancellationToken));
|
||||
}
|
||||
|
||||
// Create a new record and only set the Log field
|
||||
var attachmentUpdataRecord = new TimelineRecord() { Id = file.TimelineRecordId, Log = taskLog };
|
||||
QueueTimelineRecordUpdate(file.TimelineId, attachmentUpdataRecord);
|
||||
var logUploaded = await _jobServer.AppendLogContentAsync(_scopeIdentifier, _hubName, _planId, taskLog.Id, fs, default(CancellationToken));
|
||||
}
|
||||
else
|
||||
|
||||
// Create a new record and only set the Log field
|
||||
var attachmentUpdataRecord = new TimelineRecord() { Id = file.TimelineRecordId, Log = taskLog };
|
||||
QueueTimelineRecordUpdate(file.TimelineId, attachmentUpdataRecord);
|
||||
}
|
||||
else
|
||||
{
|
||||
// Create attachment
|
||||
using (FileStream fs = File.Open(file.Path, FileMode.Open, FileAccess.Read, FileShare.ReadWrite))
|
||||
{
|
||||
// Create attachment
|
||||
using (FileStream fs = File.Open(file.Path, FileMode.Open, FileAccess.Read, FileShare.ReadWrite))
|
||||
{
|
||||
var result = await _jobServer.CreateAttachmentAsync(_scopeIdentifier, _hubName, _planId, file.TimelineId, file.TimelineRecordId, file.Type, file.Name, fs, default(CancellationToken));
|
||||
}
|
||||
var result = await _jobServer.CreateAttachmentAsync(_scopeIdentifier, _hubName, _planId, file.TimelineId, file.TimelineRecordId, file.Type, file.Name, fs, default(CancellationToken));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -874,7 +819,7 @@ namespace GitHub.Runner.Common
|
||||
Trace.Info($"Starting to upload summary file to results service {file.Name}, {file.Path}");
|
||||
ResultsFileUploadHandler summaryHandler = async (file) =>
|
||||
{
|
||||
await _resultsServer.CreateResultsStepSummaryAsync(file.PlanId, file.JobId, file.RecordId, file.Path, CancellationToken.None);
|
||||
await _jobServer.CreateStepSummaryAsync(file.PlanId, file.JobId, file.RecordId, file.Path, CancellationToken.None);
|
||||
};
|
||||
|
||||
await UploadResultsFile(file, summaryHandler);
|
||||
@@ -885,7 +830,7 @@ namespace GitHub.Runner.Common
|
||||
Trace.Info($"Starting upload of step log file to results service {file.Name}, {file.Path}");
|
||||
ResultsFileUploadHandler stepLogHandler = async (file) =>
|
||||
{
|
||||
await _resultsServer.CreateResultsStepLogAsync(file.PlanId, file.JobId, file.RecordId, file.Path, file.Finalize, file.FirstBlock, file.TotalLines, CancellationToken.None);
|
||||
await _jobServer.CreateResultsStepLogAsync(file.PlanId, file.JobId, file.RecordId, file.Path, file.Finalize, file.FirstBlock, file.TotalLines, CancellationToken.None);
|
||||
};
|
||||
|
||||
await UploadResultsFile(file, stepLogHandler);
|
||||
@@ -896,7 +841,7 @@ namespace GitHub.Runner.Common
|
||||
Trace.Info($"Starting upload of job log file to results service {file.Name}, {file.Path}");
|
||||
ResultsFileUploadHandler jobLogHandler = async (file) =>
|
||||
{
|
||||
await _resultsServer.CreateResultsJobLogAsync(file.PlanId, file.JobId, file.Path, file.Finalize, file.FirstBlock, file.TotalLines, CancellationToken.None);
|
||||
await _jobServer.CreateResultsJobLogAsync(file.PlanId, file.JobId, file.Path, file.Finalize, file.FirstBlock, file.TotalLines, CancellationToken.None);
|
||||
};
|
||||
|
||||
await UploadResultsFile(file, jobLogHandler);
|
||||
@@ -904,11 +849,6 @@ namespace GitHub.Runner.Common
|
||||
|
||||
private async Task UploadResultsFile(ResultsUploadFileInfo file, ResultsFileUploadHandler uploadHandler)
|
||||
{
|
||||
if (!_resultsClientInitiated)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
bool uploadSucceed = false;
|
||||
try
|
||||
{
|
||||
@@ -963,6 +903,8 @@ namespace GitHub.Runner.Common
|
||||
public long TotalLines { get; set; }
|
||||
}
|
||||
|
||||
|
||||
|
||||
internal class ConsoleLineInfo
|
||||
{
|
||||
public ConsoleLineInfo(Guid recordId, string line, long? lineNumber)
|
||||
|
||||
@@ -1,42 +0,0 @@
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Linq;
|
||||
using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using GitHub.Services.Launch.Client;
|
||||
using GitHub.Services.WebApi;
|
||||
|
||||
namespace GitHub.Runner.Common
|
||||
{
|
||||
[ServiceLocator(Default = typeof(LaunchServer))]
|
||||
public interface ILaunchServer : IRunnerService
|
||||
{
|
||||
void InitializeLaunchClient(Uri uri, string token);
|
||||
|
||||
Task<ActionDownloadInfoCollection> ResolveActionsDownloadInfoAsync(Guid planId, Guid jobId, ActionReferenceList actionReferenceList, CancellationToken cancellationToken);
|
||||
}
|
||||
|
||||
public sealed class LaunchServer : RunnerService, ILaunchServer
|
||||
{
|
||||
private LaunchHttpClient _launchClient;
|
||||
|
||||
public void InitializeLaunchClient(Uri uri, string token)
|
||||
{
|
||||
var httpMessageHandler = HostContext.CreateHttpClientHandler();
|
||||
this._launchClient = new LaunchHttpClient(uri, httpMessageHandler, token, disposeHandler: true);
|
||||
}
|
||||
|
||||
public Task<ActionDownloadInfoCollection> ResolveActionsDownloadInfoAsync(Guid planId, Guid jobId, ActionReferenceList actionReferenceList,
|
||||
CancellationToken cancellationToken)
|
||||
{
|
||||
if (_launchClient != null)
|
||||
{
|
||||
return _launchClient.GetResolveActionsDownloadInfoAsync(planId, jobId, actionReferenceList,
|
||||
cancellationToken: cancellationToken);
|
||||
}
|
||||
|
||||
throw new InvalidOperationException("Launch client is not initialized.");
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,262 +0,0 @@
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Linq;
|
||||
using System.Net.Http.Headers;
|
||||
using System.Net.WebSockets;
|
||||
using System.Security;
|
||||
using System.Text;
|
||||
using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using GitHub.Runner.Sdk;
|
||||
using GitHub.Services.Common;
|
||||
using GitHub.Services.Results.Client;
|
||||
using GitHub.Services.WebApi.Utilities.Internal;
|
||||
|
||||
namespace GitHub.Runner.Common
|
||||
{
|
||||
[ServiceLocator(Default = typeof(ResultServer))]
|
||||
public interface IResultsServer : IRunnerService, IAsyncDisposable
|
||||
{
|
||||
void InitializeResultsClient(Uri uri, string liveConsoleFeedUrl, string token);
|
||||
|
||||
Task<bool> AppendLiveConsoleFeedAsync(Guid scopeIdentifier, string hubName, Guid planId, Guid timelineId, Guid timelineRecordId, Guid stepId, IList<string> lines, long? startLine, CancellationToken cancellationToken);
|
||||
|
||||
// logging and console
|
||||
Task CreateResultsStepSummaryAsync(string planId, string jobId, Guid stepId, string file,
|
||||
CancellationToken cancellationToken);
|
||||
|
||||
Task CreateResultsStepLogAsync(string planId, string jobId, Guid stepId, string file, bool finalize,
|
||||
bool firstBlock, long lineCount, CancellationToken cancellationToken);
|
||||
|
||||
Task CreateResultsJobLogAsync(string planId, string jobId, string file, bool finalize, bool firstBlock,
|
||||
long lineCount, CancellationToken cancellationToken);
|
||||
|
||||
Task UpdateResultsWorkflowStepsAsync(Guid scopeIdentifier, string hubName, Guid planId, Guid timelineId,
|
||||
IEnumerable<TimelineRecord> records, CancellationToken cancellationToken);
|
||||
}
|
||||
|
||||
public sealed class ResultServer : RunnerService, IResultsServer
|
||||
{
|
||||
private ResultsHttpClient _resultsClient;
|
||||
|
||||
private ClientWebSocket _websocketClient;
|
||||
private DateTime? _lastConnectionFailure;
|
||||
|
||||
private static readonly TimeSpan MinDelayForWebsocketReconnect = TimeSpan.FromMilliseconds(100);
|
||||
private static readonly TimeSpan MaxDelayForWebsocketReconnect = TimeSpan.FromMilliseconds(500);
|
||||
|
||||
private Task _websocketConnectTask;
|
||||
private String _liveConsoleFeedUrl;
|
||||
private string _token;
|
||||
|
||||
public void InitializeResultsClient(Uri uri, string liveConsoleFeedUrl, string token)
|
||||
{
|
||||
var httpMessageHandler = HostContext.CreateHttpClientHandler();
|
||||
this._resultsClient = new ResultsHttpClient(uri, httpMessageHandler, token, disposeHandler: true);
|
||||
_token = token;
|
||||
if (!string.IsNullOrEmpty(liveConsoleFeedUrl))
|
||||
{
|
||||
_liveConsoleFeedUrl = liveConsoleFeedUrl;
|
||||
InitializeWebsocketClient(liveConsoleFeedUrl, token, TimeSpan.Zero, retryConnection: true);
|
||||
}
|
||||
}
|
||||
|
||||
public Task CreateResultsStepSummaryAsync(string planId, string jobId, Guid stepId, string file,
|
||||
CancellationToken cancellationToken)
|
||||
{
|
||||
if (_resultsClient != null)
|
||||
{
|
||||
return _resultsClient.UploadStepSummaryAsync(planId, jobId, stepId, file,
|
||||
cancellationToken: cancellationToken);
|
||||
}
|
||||
|
||||
throw new InvalidOperationException("Results client is not initialized.");
|
||||
}
|
||||
|
||||
public Task CreateResultsStepLogAsync(string planId, string jobId, Guid stepId, string file, bool finalize,
|
||||
bool firstBlock, long lineCount, CancellationToken cancellationToken)
|
||||
{
|
||||
if (_resultsClient != null)
|
||||
{
|
||||
return _resultsClient.UploadResultsStepLogAsync(planId, jobId, stepId, file, finalize, firstBlock,
|
||||
lineCount, cancellationToken: cancellationToken);
|
||||
}
|
||||
|
||||
throw new InvalidOperationException("Results client is not initialized.");
|
||||
}
|
||||
|
||||
public Task CreateResultsJobLogAsync(string planId, string jobId, string file, bool finalize, bool firstBlock,
|
||||
long lineCount, CancellationToken cancellationToken)
|
||||
{
|
||||
if (_resultsClient != null)
|
||||
{
|
||||
return _resultsClient.UploadResultsJobLogAsync(planId, jobId, file, finalize, firstBlock, lineCount,
|
||||
cancellationToken: cancellationToken);
|
||||
}
|
||||
|
||||
throw new InvalidOperationException("Results client is not initialized.");
|
||||
}
|
||||
|
||||
public Task UpdateResultsWorkflowStepsAsync(Guid scopeIdentifier, string hubName, Guid planId, Guid timelineId,
|
||||
IEnumerable<TimelineRecord> records, CancellationToken cancellationToken)
|
||||
{
|
||||
if (_resultsClient != null)
|
||||
{
|
||||
try
|
||||
{
|
||||
var timelineRecords = records.ToList();
|
||||
return _resultsClient.UpdateWorkflowStepsAsync(planId, new List<TimelineRecord>(timelineRecords),
|
||||
cancellationToken: cancellationToken);
|
||||
}
|
||||
catch (Exception ex)
|
||||
{
|
||||
// Log error, but continue as this call is best-effort
|
||||
Trace.Info($"Failed to update steps status due to {ex.GetType().Name}");
|
||||
Trace.Error(ex);
|
||||
}
|
||||
}
|
||||
|
||||
throw new InvalidOperationException("Results client is not initialized.");
|
||||
}
|
||||
|
||||
public ValueTask DisposeAsync()
|
||||
{
|
||||
CloseWebSocket(WebSocketCloseStatus.NormalClosure, CancellationToken.None);
|
||||
|
||||
GC.SuppressFinalize(this);
|
||||
|
||||
return ValueTask.CompletedTask;
|
||||
}
|
||||
|
||||
private void InitializeWebsocketClient(string liveConsoleFeedUrl, string accessToken, TimeSpan delay, bool retryConnection = false)
|
||||
{
|
||||
if (string.IsNullOrEmpty(accessToken))
|
||||
{
|
||||
Trace.Info($"No access token from server");
|
||||
return;
|
||||
}
|
||||
|
||||
if (string.IsNullOrEmpty(liveConsoleFeedUrl))
|
||||
{
|
||||
Trace.Info($"No live console feed url from server");
|
||||
return;
|
||||
}
|
||||
|
||||
Trace.Info($"Creating websocket client ..." + liveConsoleFeedUrl);
|
||||
this._websocketClient = new ClientWebSocket();
|
||||
this._websocketClient.Options.SetRequestHeader("Authorization", $"Bearer {accessToken}");
|
||||
var userAgentValues = new List<ProductInfoHeaderValue>();
|
||||
userAgentValues.AddRange(UserAgentUtility.GetDefaultRestUserAgent());
|
||||
userAgentValues.AddRange(HostContext.UserAgents);
|
||||
this._websocketClient.Options.SetRequestHeader("User-Agent", string.Join(" ", userAgentValues.Select(x => x.ToString())));
|
||||
|
||||
// during initialization, retry upto 3 times to setup connection
|
||||
this._websocketConnectTask = ConnectWebSocketClient(liveConsoleFeedUrl, delay, retryConnection);
|
||||
}
|
||||
|
||||
private async Task ConnectWebSocketClient(string feedStreamUrl, TimeSpan delay, bool retryConnection = false)
|
||||
{
|
||||
bool connected = false;
|
||||
int retries = 0;
|
||||
|
||||
do
|
||||
{
|
||||
try
|
||||
{
|
||||
Trace.Info($"Attempting to start websocket client with delay {delay}.");
|
||||
await Task.Delay(delay);
|
||||
using var connectTimeoutTokenSource = new CancellationTokenSource(TimeSpan.FromSeconds(30));
|
||||
await this._websocketClient.ConnectAsync(new Uri(feedStreamUrl), connectTimeoutTokenSource.Token);
|
||||
Trace.Info($"Successfully started websocket client.");
|
||||
connected = true;
|
||||
}
|
||||
catch (Exception ex)
|
||||
{
|
||||
Trace.Info("Exception caught during websocket client connect, retry connection.");
|
||||
Trace.Error(ex);
|
||||
retries++;
|
||||
this._websocketClient = null;
|
||||
_lastConnectionFailure = DateTime.Now;
|
||||
}
|
||||
} while (retryConnection && !connected && retries < 3);
|
||||
}
|
||||
|
||||
public async Task<bool> AppendLiveConsoleFeedAsync(Guid scopeIdentifier, string hubName, Guid planId, Guid timelineId, Guid timelineRecordId, Guid stepId, IList<string> lines, long? startLine, CancellationToken cancellationToken)
|
||||
{
|
||||
if (_websocketConnectTask != null)
|
||||
{
|
||||
await _websocketConnectTask;
|
||||
}
|
||||
|
||||
bool delivered = false;
|
||||
int retries = 0;
|
||||
|
||||
// "_websocketClient != null" implies either: We have a successful connection OR we have to attempt sending again and then reconnect
|
||||
// ...in other words, if websocket client is null, we will skip sending to websocket
|
||||
if (_websocketClient != null)
|
||||
{
|
||||
var linesWrapper = startLine.HasValue
|
||||
? new TimelineRecordFeedLinesWrapper(stepId, lines, startLine.Value)
|
||||
: new TimelineRecordFeedLinesWrapper(stepId, lines);
|
||||
var jsonData = StringUtil.ConvertToJson(linesWrapper);
|
||||
var jsonDataBytes = Encoding.UTF8.GetBytes(jsonData);
|
||||
// break the message into chunks of 1024 bytes
|
||||
for (var i = 0; i < jsonDataBytes.Length; i += 1 * 1024)
|
||||
{
|
||||
var lastChunk = i + (1 * 1024) >= jsonDataBytes.Length;
|
||||
var chunk = new ArraySegment<byte>(jsonDataBytes, i, Math.Min(1 * 1024, jsonDataBytes.Length - i));
|
||||
|
||||
delivered = false;
|
||||
while (!delivered && retries < 3)
|
||||
{
|
||||
try
|
||||
{
|
||||
if (_websocketClient != null)
|
||||
{
|
||||
await _websocketClient.SendAsync(chunk, WebSocketMessageType.Text, endOfMessage: lastChunk, cancellationToken);
|
||||
delivered = true;
|
||||
}
|
||||
}
|
||||
catch (Exception ex)
|
||||
{
|
||||
var delay = BackoffTimerHelper.GetRandomBackoff(MinDelayForWebsocketReconnect, MaxDelayForWebsocketReconnect);
|
||||
Trace.Info($"Websocket is not open, let's attempt to connect back again with random backoff {delay} ms.");
|
||||
Trace.Verbose(ex.ToString());
|
||||
retries++;
|
||||
InitializeWebsocketClient(_liveConsoleFeedUrl, _token, delay);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!delivered)
|
||||
{
|
||||
// Giving up for now, so next invocation of this method won't attempt to reconnect
|
||||
_websocketClient = null;
|
||||
|
||||
// however if 10 minutes have already passed, let's try reestablish connection again
|
||||
if (_lastConnectionFailure.HasValue && DateTime.Now > _lastConnectionFailure.Value.AddMinutes(10))
|
||||
{
|
||||
// Some minutes passed since we retried last time, try connection again
|
||||
InitializeWebsocketClient(_liveConsoleFeedUrl, _token, TimeSpan.Zero);
|
||||
}
|
||||
}
|
||||
|
||||
return delivered;
|
||||
}
|
||||
|
||||
private void CloseWebSocket(WebSocketCloseStatus closeStatus, CancellationToken cancellationToken)
|
||||
{
|
||||
try
|
||||
{
|
||||
_websocketClient?.CloseOutputAsync(closeStatus, "Closing websocket", cancellationToken);
|
||||
}
|
||||
catch (Exception websocketEx)
|
||||
{
|
||||
// In some cases this might be okay since the websocket might be open yet, so just close and don't trace exceptions
|
||||
Trace.Info($"Failed to close websocket gracefully {websocketEx.GetType().Name}");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -19,14 +19,7 @@ namespace GitHub.Runner.Common
|
||||
|
||||
Task<AgentJobRequestMessage> GetJobMessageAsync(string id, CancellationToken token);
|
||||
|
||||
Task CompleteJobAsync(
|
||||
Guid planId,
|
||||
Guid jobId,
|
||||
TaskResult result,
|
||||
Dictionary<String, VariableValue> outputs,
|
||||
IList<StepResult> stepResults,
|
||||
IList<Annotation> jobAnnotations,
|
||||
CancellationToken token);
|
||||
Task CompleteJobAsync(Guid planId, Guid jobId, TaskResult result, Dictionary<String, VariableValue> outputs, IList<StepResult> stepResults, CancellationToken token);
|
||||
|
||||
Task<RenewJobResponse> RenewJobAsync(Guid planId, Guid jobId, CancellationToken token);
|
||||
}
|
||||
@@ -58,30 +51,34 @@ namespace GitHub.Runner.Common
|
||||
public Task<AgentJobRequestMessage> GetJobMessageAsync(string id, CancellationToken cancellationToken)
|
||||
{
|
||||
CheckConnection();
|
||||
return RetryRequest<AgentJobRequestMessage>(
|
||||
async () => await _runServiceHttpClient.GetJobMessageAsync(requestUri, id, cancellationToken), cancellationToken,
|
||||
shouldRetry: ex => ex is not TaskOrchestrationJobAlreadyAcquiredException);
|
||||
var jobMessage = RetryRequest<AgentJobRequestMessage>(
|
||||
async () => await _runServiceHttpClient.GetJobMessageAsync(requestUri, id, cancellationToken), cancellationToken);
|
||||
if (jobMessage == null)
|
||||
{
|
||||
throw new TaskOrchestrationJobNotFoundException(id);
|
||||
}
|
||||
|
||||
return jobMessage;
|
||||
}
|
||||
|
||||
public Task CompleteJobAsync(
|
||||
Guid planId,
|
||||
Guid jobId,
|
||||
TaskResult result,
|
||||
Dictionary<String, VariableValue> outputs,
|
||||
IList<StepResult> stepResults,
|
||||
IList<Annotation> jobAnnotations,
|
||||
CancellationToken cancellationToken)
|
||||
public Task CompleteJobAsync(Guid planId, Guid jobId, TaskResult result, Dictionary<String, VariableValue> outputs, IList<StepResult> stepResults, CancellationToken cancellationToken)
|
||||
{
|
||||
CheckConnection();
|
||||
return RetryRequest(
|
||||
async () => await _runServiceHttpClient.CompleteJobAsync(requestUri, planId, jobId, result, outputs, stepResults, jobAnnotations, cancellationToken), cancellationToken);
|
||||
async () => await _runServiceHttpClient.CompleteJobAsync(requestUri, planId, jobId, result, outputs, stepResults, cancellationToken), cancellationToken);
|
||||
}
|
||||
|
||||
public Task<RenewJobResponse> RenewJobAsync(Guid planId, Guid jobId, CancellationToken cancellationToken)
|
||||
{
|
||||
CheckConnection();
|
||||
return RetryRequest<RenewJobResponse>(
|
||||
var renewJobResponse = RetryRequest<RenewJobResponse>(
|
||||
async () => await _runServiceHttpClient.RenewJobAsync(requestUri, planId, jobId, cancellationToken), cancellationToken);
|
||||
if (renewJobResponse == null)
|
||||
{
|
||||
throw new TaskOrchestrationJobNotFoundException(jobId.ToString());
|
||||
}
|
||||
|
||||
return renewJobResponse;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,237 +0,0 @@
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
using GitHub.Services.WebApi;
|
||||
using GitHub.Services.Common;
|
||||
using GitHub.Runner.Sdk;
|
||||
using System.Net.Http;
|
||||
using System.Net.Http.Headers;
|
||||
using System.Linq;
|
||||
|
||||
namespace GitHub.Runner.Common
|
||||
{
|
||||
[ServiceLocator(Default = typeof(RunnerDotcomServer))]
|
||||
public interface IRunnerDotcomServer : IRunnerService
|
||||
{
|
||||
Task<List<TaskAgent>> GetRunnersAsync(int runnerGroupId, string githubUrl, string githubToken, string agentName);
|
||||
|
||||
Task<DistributedTask.WebApi.Runner> AddRunnerAsync(int runnerGroupId, TaskAgent agent, string githubUrl, string githubToken, string publicKey);
|
||||
Task<List<TaskAgentPool>> GetRunnerGroupsAsync(string githubUrl, string githubToken);
|
||||
|
||||
string GetGitHubRequestId(HttpResponseHeaders headers);
|
||||
}
|
||||
|
||||
public enum RequestType
|
||||
{
|
||||
Get,
|
||||
Post,
|
||||
Patch,
|
||||
Delete
|
||||
}
|
||||
|
||||
public class RunnerDotcomServer : RunnerService, IRunnerDotcomServer
|
||||
{
|
||||
private ITerminal _term;
|
||||
|
||||
public override void Initialize(IHostContext hostContext)
|
||||
{
|
||||
base.Initialize(hostContext);
|
||||
_term = hostContext.GetService<ITerminal>();
|
||||
}
|
||||
|
||||
|
||||
public async Task<List<TaskAgent>> GetRunnersAsync(int runnerGroupId, string githubUrl, string githubToken, string agentName = null)
|
||||
{
|
||||
var githubApiUrl = "";
|
||||
var gitHubUrlBuilder = new UriBuilder(githubUrl);
|
||||
var path = gitHubUrlBuilder.Path.Split('/', '\\', StringSplitOptions.RemoveEmptyEntries);
|
||||
if (path.Length == 1)
|
||||
{
|
||||
// org runner
|
||||
if (UrlUtil.IsHostedServer(gitHubUrlBuilder))
|
||||
{
|
||||
githubApiUrl = $"{gitHubUrlBuilder.Scheme}://api.{gitHubUrlBuilder.Host}/orgs/{path[0]}/actions/runner-groups/{runnerGroupId}/runners";
|
||||
}
|
||||
else
|
||||
{
|
||||
githubApiUrl = $"{gitHubUrlBuilder.Scheme}://{gitHubUrlBuilder.Host}/api/v3/orgs/{path[0]}/actions/runner-groups/{runnerGroupId}/runners";
|
||||
}
|
||||
}
|
||||
else if (path.Length == 2)
|
||||
{
|
||||
// repo or enterprise runner.
|
||||
if (!string.Equals(path[0], "enterprises", StringComparison.OrdinalIgnoreCase))
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
if (UrlUtil.IsHostedServer(gitHubUrlBuilder))
|
||||
{
|
||||
githubApiUrl = $"{gitHubUrlBuilder.Scheme}://api.{gitHubUrlBuilder.Host}/{path[0]}/{path[1]}/actions/runner-groups/{runnerGroupId}/runners";
|
||||
}
|
||||
else
|
||||
{
|
||||
githubApiUrl = $"{gitHubUrlBuilder.Scheme}://{gitHubUrlBuilder.Host}/api/v3/{path[0]}/{path[1]}/actions/runner-groups/{runnerGroupId}/runners";
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
throw new ArgumentException($"'{githubUrl}' should point to an org or enterprise.");
|
||||
}
|
||||
|
||||
var runnersList = await RetryRequest<ListRunnersResponse>(githubApiUrl, githubToken, RequestType.Get, 3, "Failed to get agents pools");
|
||||
var agents = runnersList.ToTaskAgents();
|
||||
|
||||
if (string.IsNullOrEmpty(agentName))
|
||||
{
|
||||
return agents;
|
||||
}
|
||||
|
||||
return agents.Where(x => string.Equals(x.Name, agentName, StringComparison.OrdinalIgnoreCase)).ToList();
|
||||
}
|
||||
|
||||
public async Task<List<TaskAgentPool>> GetRunnerGroupsAsync(string githubUrl, string githubToken)
|
||||
{
|
||||
var githubApiUrl = "";
|
||||
var gitHubUrlBuilder = new UriBuilder(githubUrl);
|
||||
var path = gitHubUrlBuilder.Path.Split('/', '\\', StringSplitOptions.RemoveEmptyEntries);
|
||||
if (path.Length == 1)
|
||||
{
|
||||
// org runner
|
||||
if (UrlUtil.IsHostedServer(gitHubUrlBuilder))
|
||||
{
|
||||
githubApiUrl = $"{gitHubUrlBuilder.Scheme}://api.{gitHubUrlBuilder.Host}/orgs/{path[0]}/actions/runner-groups";
|
||||
}
|
||||
else
|
||||
{
|
||||
githubApiUrl = $"{gitHubUrlBuilder.Scheme}://{gitHubUrlBuilder.Host}/api/v3/orgs/{path[0]}/actions/runner-groups";
|
||||
}
|
||||
}
|
||||
else if (path.Length == 2)
|
||||
{
|
||||
// repo or enterprise runner.
|
||||
if (!string.Equals(path[0], "enterprises", StringComparison.OrdinalIgnoreCase))
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
if (UrlUtil.IsHostedServer(gitHubUrlBuilder))
|
||||
{
|
||||
githubApiUrl = $"{gitHubUrlBuilder.Scheme}://api.{gitHubUrlBuilder.Host}/{path[0]}/{path[1]}/actions/runner-groups";
|
||||
}
|
||||
else
|
||||
{
|
||||
githubApiUrl = $"{gitHubUrlBuilder.Scheme}://{gitHubUrlBuilder.Host}/api/v3/{path[0]}/{path[1]}/actions/runner-groups";
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
throw new ArgumentException($"'{githubUrl}' should point to an org or enterprise.");
|
||||
}
|
||||
|
||||
var agentPools = await RetryRequest<RunnerGroupList>(githubApiUrl, githubToken, RequestType.Get, 3, "Failed to get agents pools");
|
||||
|
||||
return agentPools?.ToAgentPoolList();
|
||||
}
|
||||
|
||||
public async Task<DistributedTask.WebApi.Runner> AddRunnerAsync(int runnerGroupId, TaskAgent agent, string githubUrl, string githubToken, string publicKey)
|
||||
{
|
||||
var gitHubUrlBuilder = new UriBuilder(githubUrl);
|
||||
var path = gitHubUrlBuilder.Path.Split('/', '\\', StringSplitOptions.RemoveEmptyEntries);
|
||||
string githubApiUrl;
|
||||
if (UrlUtil.IsHostedServer(gitHubUrlBuilder))
|
||||
{
|
||||
githubApiUrl = $"{gitHubUrlBuilder.Scheme}://api.{gitHubUrlBuilder.Host}/actions/runners/register";
|
||||
}
|
||||
else
|
||||
{
|
||||
githubApiUrl = $"{gitHubUrlBuilder.Scheme}://{gitHubUrlBuilder.Host}/api/v3/actions/runners/register";
|
||||
}
|
||||
|
||||
var bodyObject = new Dictionary<string, Object>()
|
||||
{
|
||||
{"url", githubUrl},
|
||||
{"group_id", runnerGroupId},
|
||||
{"name", agent.Name},
|
||||
{"version", agent.Version},
|
||||
{"updates_disabled", agent.DisableUpdate},
|
||||
{"ephemeral", agent.Ephemeral},
|
||||
{"labels", agent.Labels},
|
||||
{"public_key", publicKey}
|
||||
};
|
||||
|
||||
var body = new StringContent(StringUtil.ConvertToJson(bodyObject), null, "application/json");
|
||||
|
||||
return await RetryRequest<DistributedTask.WebApi.Runner>(githubApiUrl, githubToken, RequestType.Post, 3, "Failed to add agent", body);
|
||||
}
|
||||
|
||||
private async Task<T> RetryRequest<T>(string githubApiUrl, string githubToken, RequestType requestType, int maxRetryAttemptsCount = 5, string errorMessage = null, StringContent body = null)
|
||||
{
|
||||
int retry = 0;
|
||||
while (true)
|
||||
{
|
||||
retry++;
|
||||
using (var httpClientHandler = HostContext.CreateHttpClientHandler())
|
||||
using (var httpClient = new HttpClient(httpClientHandler))
|
||||
{
|
||||
httpClient.DefaultRequestHeaders.Authorization = new AuthenticationHeaderValue("RemoteAuth", githubToken);
|
||||
httpClient.DefaultRequestHeaders.UserAgent.AddRange(HostContext.UserAgents);
|
||||
|
||||
var responseStatus = System.Net.HttpStatusCode.OK;
|
||||
try
|
||||
{
|
||||
HttpResponseMessage response = null;
|
||||
if (requestType == RequestType.Get)
|
||||
{
|
||||
response = await httpClient.GetAsync(githubApiUrl);
|
||||
}
|
||||
else
|
||||
{
|
||||
response = await httpClient.PostAsync(githubApiUrl, body);
|
||||
}
|
||||
|
||||
if (response != null)
|
||||
{
|
||||
responseStatus = response.StatusCode;
|
||||
var githubRequestId = GetGitHubRequestId(response.Headers);
|
||||
|
||||
if (response.IsSuccessStatusCode)
|
||||
{
|
||||
Trace.Info($"Http response code: {response.StatusCode} from '{requestType.ToString()} {githubApiUrl}' ({githubRequestId})");
|
||||
var jsonResponse = await response.Content.ReadAsStringAsync();
|
||||
return StringUtil.ConvertFromJson<T>(jsonResponse);
|
||||
}
|
||||
else
|
||||
{
|
||||
_term.WriteError($"Http response code: {response.StatusCode} from '{requestType.ToString()} {githubApiUrl}' (Request Id: {githubRequestId})");
|
||||
var errorResponse = await response.Content.ReadAsStringAsync();
|
||||
_term.WriteError(errorResponse);
|
||||
response.EnsureSuccessStatusCode();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
catch (Exception ex) when (retry < maxRetryAttemptsCount && responseStatus != System.Net.HttpStatusCode.NotFound)
|
||||
{
|
||||
Trace.Error($"{errorMessage} -- Atempt: {retry}");
|
||||
Trace.Error(ex);
|
||||
}
|
||||
}
|
||||
var backOff = BackoffTimerHelper.GetRandomBackoff(TimeSpan.FromSeconds(1), TimeSpan.FromSeconds(5));
|
||||
Trace.Info($"Retrying in {backOff.Seconds} seconds");
|
||||
await Task.Delay(backOff);
|
||||
}
|
||||
}
|
||||
|
||||
public string GetGitHubRequestId(HttpResponseHeaders headers)
|
||||
{
|
||||
if (headers.TryGetValues("x-github-request-id", out var headerValues))
|
||||
{
|
||||
return headerValues.FirstOrDefault();
|
||||
}
|
||||
return string.Empty;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,4 +1,4 @@
|
||||
using System;
|
||||
using System;
|
||||
using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
using GitHub.Runner.Sdk;
|
||||
@@ -83,8 +83,7 @@ namespace GitHub.Runner.Common
|
||||
|
||||
protected async Task<T> RetryRequest<T>(Func<Task<T>> func,
|
||||
CancellationToken cancellationToken,
|
||||
int maxRetryAttemptsCount = 5,
|
||||
Func<Exception, bool> shouldRetry = null
|
||||
int maxRetryAttemptsCount = 5
|
||||
)
|
||||
{
|
||||
var retryCount = 0;
|
||||
@@ -97,7 +96,7 @@ namespace GitHub.Runner.Common
|
||||
return await func();
|
||||
}
|
||||
// TODO: Add handling of non-retriable exceptions: https://github.com/github/actions-broker/issues/122
|
||||
catch (Exception ex) when (retryCount < maxRetryAttemptsCount && (shouldRetry == null || shouldRetry(ex)))
|
||||
catch (Exception ex) when (retryCount < maxRetryAttemptsCount)
|
||||
{
|
||||
Trace.Error("Catch exception during request");
|
||||
Trace.Error(ex);
|
||||
|
||||
@@ -1,209 +0,0 @@
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Diagnostics;
|
||||
using System.IO;
|
||||
using System.Runtime.InteropServices;
|
||||
using System.Security.Cryptography;
|
||||
using System.Text;
|
||||
using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using GitHub.Runner.Common;
|
||||
using GitHub.Runner.Listener.Configuration;
|
||||
using GitHub.Runner.Sdk;
|
||||
using GitHub.Services.Common;
|
||||
using GitHub.Runner.Common.Util;
|
||||
using GitHub.Services.OAuth;
|
||||
|
||||
namespace GitHub.Runner.Listener
|
||||
{
|
||||
public sealed class BrokerMessageListener : RunnerService, IMessageListener
|
||||
{
|
||||
private RunnerSettings _settings;
|
||||
private ITerminal _term;
|
||||
private TimeSpan _getNextMessageRetryInterval;
|
||||
private TaskAgentStatus runnerStatus = TaskAgentStatus.Online;
|
||||
private CancellationTokenSource _getMessagesTokenSource;
|
||||
private IBrokerServer _brokerServer;
|
||||
|
||||
public override void Initialize(IHostContext hostContext)
|
||||
{
|
||||
base.Initialize(hostContext);
|
||||
|
||||
_term = HostContext.GetService<ITerminal>();
|
||||
_brokerServer = HostContext.GetService<IBrokerServer>();
|
||||
}
|
||||
|
||||
public async Task<Boolean> CreateSessionAsync(CancellationToken token)
|
||||
{
|
||||
await RefreshBrokerConnection();
|
||||
return await Task.FromResult(true);
|
||||
}
|
||||
|
||||
public async Task DeleteSessionAsync()
|
||||
{
|
||||
await Task.CompletedTask;
|
||||
}
|
||||
|
||||
public void OnJobStatus(object sender, JobStatusEventArgs e)
|
||||
{
|
||||
Trace.Info("Received job status event. JobState: {0}", e.Status);
|
||||
runnerStatus = e.Status;
|
||||
try
|
||||
{
|
||||
_getMessagesTokenSource?.Cancel();
|
||||
}
|
||||
catch (ObjectDisposedException)
|
||||
{
|
||||
Trace.Info("_getMessagesTokenSource is already disposed.");
|
||||
}
|
||||
}
|
||||
|
||||
public async Task<TaskAgentMessage> GetNextMessageAsync(CancellationToken token)
|
||||
{
|
||||
bool encounteringError = false;
|
||||
int continuousError = 0;
|
||||
Stopwatch heartbeat = new();
|
||||
heartbeat.Restart();
|
||||
var maxRetryCount = 10;
|
||||
|
||||
while (true)
|
||||
{
|
||||
TaskAgentMessage message = null;
|
||||
_getMessagesTokenSource = CancellationTokenSource.CreateLinkedTokenSource(token);
|
||||
try
|
||||
{
|
||||
message = await _brokerServer.GetRunnerMessageAsync(_getMessagesTokenSource.Token, runnerStatus, BuildConstants.RunnerPackage.Version);
|
||||
|
||||
if (message == null)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
|
||||
return message;
|
||||
}
|
||||
catch (OperationCanceledException) when (_getMessagesTokenSource.Token.IsCancellationRequested && !token.IsCancellationRequested)
|
||||
{
|
||||
Trace.Info("Get messages has been cancelled using local token source. Continue to get messages with new status.");
|
||||
continue;
|
||||
}
|
||||
catch (OperationCanceledException) when (token.IsCancellationRequested)
|
||||
{
|
||||
Trace.Info("Get next message has been cancelled.");
|
||||
throw;
|
||||
}
|
||||
catch (TaskAgentAccessTokenExpiredException)
|
||||
{
|
||||
Trace.Info("Runner OAuth token has been revoked. Unable to pull message.");
|
||||
throw;
|
||||
}
|
||||
catch (AccessDeniedException e) when (e.InnerException is InvalidTaskAgentVersionException)
|
||||
{
|
||||
throw;
|
||||
}
|
||||
catch (Exception ex)
|
||||
{
|
||||
Trace.Error("Catch exception during get next message.");
|
||||
Trace.Error(ex);
|
||||
|
||||
if (!IsGetNextMessageExceptionRetriable(ex))
|
||||
{
|
||||
throw;
|
||||
}
|
||||
else
|
||||
{
|
||||
continuousError++;
|
||||
//retry after a random backoff to avoid service throttling
|
||||
//in case of there is a service error happened and all agents get kicked off of the long poll and all agent try to reconnect back at the same time.
|
||||
if (continuousError <= 5)
|
||||
{
|
||||
// random backoff [15, 30]
|
||||
_getNextMessageRetryInterval = BackoffTimerHelper.GetRandomBackoff(TimeSpan.FromSeconds(15), TimeSpan.FromSeconds(30), _getNextMessageRetryInterval);
|
||||
}
|
||||
else if (continuousError >= maxRetryCount)
|
||||
{
|
||||
throw;
|
||||
}
|
||||
else
|
||||
{
|
||||
// more aggressive backoff [30, 60]
|
||||
_getNextMessageRetryInterval = BackoffTimerHelper.GetRandomBackoff(TimeSpan.FromSeconds(30), TimeSpan.FromSeconds(60), _getNextMessageRetryInterval);
|
||||
}
|
||||
|
||||
if (!encounteringError)
|
||||
{
|
||||
//print error only on the first consecutive error
|
||||
_term.WriteError($"{DateTime.UtcNow:u}: Runner connect error: {ex.Message}. Retrying until reconnected.");
|
||||
encounteringError = true;
|
||||
}
|
||||
|
||||
// re-create VssConnection before next retry
|
||||
await RefreshBrokerConnection();
|
||||
|
||||
Trace.Info("Sleeping for {0} seconds before retrying.", _getNextMessageRetryInterval.TotalSeconds);
|
||||
await HostContext.Delay(_getNextMessageRetryInterval, token);
|
||||
}
|
||||
}
|
||||
finally
|
||||
{
|
||||
_getMessagesTokenSource.Dispose();
|
||||
}
|
||||
|
||||
if (message == null)
|
||||
{
|
||||
if (heartbeat.Elapsed > TimeSpan.FromMinutes(30))
|
||||
{
|
||||
Trace.Info($"No message retrieved within last 30 minutes.");
|
||||
heartbeat.Restart();
|
||||
}
|
||||
else
|
||||
{
|
||||
Trace.Verbose($"No message retrieved.");
|
||||
}
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
Trace.Info($"Message '{message.MessageId}' received.");
|
||||
}
|
||||
}
|
||||
|
||||
public async Task DeleteMessageAsync(TaskAgentMessage message)
|
||||
{
|
||||
await Task.CompletedTask;
|
||||
}
|
||||
|
||||
private bool IsGetNextMessageExceptionRetriable(Exception ex)
|
||||
{
|
||||
if (ex is TaskAgentNotFoundException ||
|
||||
ex is TaskAgentPoolNotFoundException ||
|
||||
ex is TaskAgentSessionExpiredException ||
|
||||
ex is AccessDeniedException ||
|
||||
ex is VssUnauthorizedException)
|
||||
{
|
||||
Trace.Info($"Non-retriable exception: {ex.Message}");
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
Trace.Info($"Retriable exception: {ex.Message}");
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
private async Task RefreshBrokerConnection()
|
||||
{
|
||||
var configManager = HostContext.GetService<IConfigurationManager>();
|
||||
_settings = configManager.LoadSettings();
|
||||
|
||||
if (_settings.ServerUrlV2 == null)
|
||||
{
|
||||
throw new InvalidOperationException("ServerUrlV2 is not set");
|
||||
}
|
||||
|
||||
var credMgr = HostContext.GetService<ICredentialManager>();
|
||||
VssCredentials creds = credMgr.LoadCredentials();
|
||||
await _brokerServer.ConnectAsync(new Uri(_settings.ServerUrlV2), creds);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -38,7 +38,6 @@ namespace GitHub.Runner.Listener
|
||||
Constants.Runner.CommandLine.Flags.Replace,
|
||||
Constants.Runner.CommandLine.Flags.RunAsService,
|
||||
Constants.Runner.CommandLine.Flags.Unattended,
|
||||
Constants.Runner.CommandLine.Flags.NoDefaultLabels,
|
||||
Constants.Runner.CommandLine.Args.Auth,
|
||||
Constants.Runner.CommandLine.Args.Labels,
|
||||
Constants.Runner.CommandLine.Args.MonitorSocketAddress,
|
||||
@@ -86,7 +85,6 @@ namespace GitHub.Runner.Listener
|
||||
public bool Ephemeral => TestFlag(Constants.Runner.CommandLine.Flags.Ephemeral);
|
||||
public bool GenerateServiceConfig => TestFlag(Constants.Runner.CommandLine.Flags.GenerateServiceConfig);
|
||||
public bool Help => TestFlag(Constants.Runner.CommandLine.Flags.Help);
|
||||
public bool NoDefaultLabels => TestFlag(Constants.Runner.CommandLine.Flags.NoDefaultLabels);
|
||||
public bool Unattended => TestFlag(Constants.Runner.CommandLine.Flags.Unattended);
|
||||
public bool Version => TestFlag(Constants.Runner.CommandLine.Flags.Version);
|
||||
public bool RemoveLocalConfig => TestFlag(Constants.Runner.CommandLine.Flags.Local);
|
||||
|
||||
@@ -31,14 +31,12 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
{
|
||||
private IConfigurationStore _store;
|
||||
private IRunnerServer _runnerServer;
|
||||
private IRunnerDotcomServer _dotcomServer;
|
||||
private ITerminal _term;
|
||||
|
||||
public override void Initialize(IHostContext hostContext)
|
||||
{
|
||||
base.Initialize(hostContext);
|
||||
_runnerServer = HostContext.GetService<IRunnerServer>();
|
||||
_dotcomServer = HostContext.GetService<IRunnerDotcomServer>();
|
||||
Trace.Verbose("Creating _store");
|
||||
_store = hostContext.GetService<IConfigurationStore>();
|
||||
Trace.Verbose("store created");
|
||||
@@ -115,7 +113,6 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
ICredentialProvider credProvider = null;
|
||||
VssCredentials creds = null;
|
||||
_term.WriteSection("Authentication");
|
||||
string registerToken = string.Empty;
|
||||
while (true)
|
||||
{
|
||||
// When testing against a dev deployment of Actions Service, set this environment variable
|
||||
@@ -133,11 +130,9 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
else
|
||||
{
|
||||
runnerSettings.GitHubUrl = inputUrl;
|
||||
registerToken = await GetRunnerTokenAsync(command, inputUrl, "registration");
|
||||
var registerToken = await GetRunnerTokenAsync(command, inputUrl, "registration");
|
||||
GitHubAuthResult authResult = await GetTenantCredential(inputUrl, registerToken, Constants.RunnerEvent.Register);
|
||||
runnerSettings.ServerUrl = authResult.TenantUrl;
|
||||
runnerSettings.UseV2Flow = authResult.UseV2Flow;
|
||||
_term.WriteLine($"Using V2 flow: {runnerSettings.UseV2Flow}");
|
||||
creds = authResult.ToVssCredentials();
|
||||
Trace.Info("cred retrieved via GitHub auth");
|
||||
}
|
||||
@@ -181,11 +176,9 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
// We want to use the native CSP of the platform for storage, so we use the RSACSP directly
|
||||
RSAParameters publicKey;
|
||||
var keyManager = HostContext.GetService<IRSAKeyManager>();
|
||||
string publicKeyXML;
|
||||
using (var rsa = keyManager.CreateKey())
|
||||
{
|
||||
publicKey = rsa.ExportParameters(false);
|
||||
publicKeyXML = rsa.ToXmlString(includePrivateParameters: false);
|
||||
}
|
||||
|
||||
_term.WriteSection("Runner Registration");
|
||||
@@ -193,17 +186,9 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
// If we have more than one runner group available, allow the user to specify which one to be added into
|
||||
string poolName = null;
|
||||
TaskAgentPool agentPool = null;
|
||||
List<TaskAgentPool> agentPools;
|
||||
if (runnerSettings.UseV2Flow)
|
||||
{
|
||||
agentPools = await _dotcomServer.GetRunnerGroupsAsync(runnerSettings.GitHubUrl, registerToken);
|
||||
}
|
||||
else
|
||||
{
|
||||
agentPools = await _runnerServer.GetAgentPoolsAsync();
|
||||
}
|
||||
|
||||
List<TaskAgentPool> agentPools = await _runnerServer.GetAgentPoolsAsync();
|
||||
TaskAgentPool defaultPool = agentPools?.Where(x => x.IsInternal).FirstOrDefault();
|
||||
|
||||
if (agentPools?.Where(x => !x.IsHosted).Count() > 0)
|
||||
{
|
||||
poolName = command.GetRunnerGroupName(defaultPool?.Name);
|
||||
@@ -241,16 +226,8 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
|
||||
var userLabels = command.GetLabels();
|
||||
_term.WriteLine();
|
||||
List<TaskAgent> agents;
|
||||
if (runnerSettings.UseV2Flow)
|
||||
{
|
||||
agents = await _dotcomServer.GetRunnersAsync(runnerSettings.PoolId, runnerSettings.GitHubUrl, registerToken, runnerSettings.AgentName);
|
||||
}
|
||||
else
|
||||
{
|
||||
agents = await _runnerServer.GetAgentsAsync(runnerSettings.PoolId, runnerSettings.AgentName);
|
||||
}
|
||||
|
||||
var agents = await _runnerServer.GetAgentsAsync(runnerSettings.PoolId, runnerSettings.AgentName);
|
||||
Trace.Verbose("Returns {0} agents", agents.Count);
|
||||
agent = agents.FirstOrDefault();
|
||||
if (agent != null)
|
||||
@@ -259,7 +236,7 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
if (command.GetReplace())
|
||||
{
|
||||
// Update existing agent with new PublicKey, agent version.
|
||||
agent = UpdateExistingAgent(agent, publicKey, userLabels, runnerSettings.Ephemeral, command.DisableUpdate, command.NoDefaultLabels);
|
||||
agent = UpdateExistingAgent(agent, publicKey, userLabels, runnerSettings.Ephemeral, command.DisableUpdate);
|
||||
|
||||
try
|
||||
{
|
||||
@@ -293,27 +270,11 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
else
|
||||
{
|
||||
// Create a new agent.
|
||||
agent = CreateNewAgent(runnerSettings.AgentName, publicKey, userLabels, runnerSettings.Ephemeral, command.DisableUpdate, command.NoDefaultLabels);
|
||||
agent = CreateNewAgent(runnerSettings.AgentName, publicKey, userLabels, runnerSettings.Ephemeral, command.DisableUpdate);
|
||||
|
||||
try
|
||||
{
|
||||
if (runnerSettings.UseV2Flow)
|
||||
{
|
||||
var runner = await _dotcomServer.AddRunnerAsync(runnerSettings.PoolId, agent, runnerSettings.GitHubUrl, registerToken, publicKeyXML);
|
||||
runnerSettings.ServerUrlV2 = runner.RunnerAuthorization.ServerUrl;
|
||||
|
||||
agent.Id = runner.Id;
|
||||
agent.Authorization = new TaskAgentAuthorization()
|
||||
{
|
||||
AuthorizationUrl = runner.RunnerAuthorization.AuthorizationUrl,
|
||||
ClientId = new Guid(runner.RunnerAuthorization.ClientId)
|
||||
};
|
||||
}
|
||||
else
|
||||
{
|
||||
agent = await _runnerServer.AddAgentAsync(runnerSettings.PoolId, agent);
|
||||
}
|
||||
|
||||
agent = await _runnerServer.AddAgentAsync(runnerSettings.PoolId, agent);
|
||||
if (command.DisableUpdate &&
|
||||
command.DisableUpdate != agent.DisableUpdate)
|
||||
{
|
||||
@@ -364,28 +325,24 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
}
|
||||
|
||||
// Testing agent connection, detect any potential connection issue, like local clock skew that cause OAuth token expired.
|
||||
|
||||
if (!runnerSettings.UseV2Flow)
|
||||
var credMgr = HostContext.GetService<ICredentialManager>();
|
||||
VssCredentials credential = credMgr.LoadCredentials();
|
||||
try
|
||||
{
|
||||
var credMgr = HostContext.GetService<ICredentialManager>();
|
||||
VssCredentials credential = credMgr.LoadCredentials();
|
||||
try
|
||||
{
|
||||
await _runnerServer.ConnectAsync(new Uri(runnerSettings.ServerUrl), credential);
|
||||
// ConnectAsync() hits _apis/connectionData which is an anonymous endpoint
|
||||
// Need to hit an authenticate endpoint to trigger OAuth token exchange.
|
||||
await _runnerServer.GetAgentPoolsAsync();
|
||||
_term.WriteSuccessMessage("Runner connection is good");
|
||||
}
|
||||
catch (VssOAuthTokenRequestException ex) when (ex.Message.Contains("Current server time is"))
|
||||
{
|
||||
// there are two exception messages server send that indicate clock skew.
|
||||
// 1. The bearer token expired on {jwt.ValidTo}. Current server time is {DateTime.UtcNow}.
|
||||
// 2. The bearer token is not valid until {jwt.ValidFrom}. Current server time is {DateTime.UtcNow}.
|
||||
Trace.Error("Catch exception during test agent connection.");
|
||||
Trace.Error(ex);
|
||||
throw new Exception("The local machine's clock may be out of sync with the server time by more than five minutes. Please sync your clock with your domain or internet time and try again.");
|
||||
}
|
||||
await _runnerServer.ConnectAsync(new Uri(runnerSettings.ServerUrl), credential);
|
||||
// ConnectAsync() hits _apis/connectionData which is an anonymous endpoint
|
||||
// Need to hit an authenticate endpoint to trigger OAuth token exchange.
|
||||
await _runnerServer.GetAgentPoolsAsync();
|
||||
_term.WriteSuccessMessage("Runner connection is good");
|
||||
}
|
||||
catch (VssOAuthTokenRequestException ex) when (ex.Message.Contains("Current server time is"))
|
||||
{
|
||||
// there are two exception messages server send that indicate clock skew.
|
||||
// 1. The bearer token expired on {jwt.ValidTo}. Current server time is {DateTime.UtcNow}.
|
||||
// 2. The bearer token is not valid until {jwt.ValidFrom}. Current server time is {DateTime.UtcNow}.
|
||||
Trace.Error("Catch exception during test agent connection.");
|
||||
Trace.Error(ex);
|
||||
throw new Exception("The local machine's clock may be out of sync with the server time by more than five minutes. Please sync your clock with your domain or internet time and try again.");
|
||||
}
|
||||
|
||||
_term.WriteSection("Runner settings");
|
||||
@@ -554,7 +511,7 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
}
|
||||
|
||||
|
||||
private TaskAgent UpdateExistingAgent(TaskAgent agent, RSAParameters publicKey, ISet<string> userLabels, bool ephemeral, bool disableUpdate, bool noDefaultLabels)
|
||||
private TaskAgent UpdateExistingAgent(TaskAgent agent, RSAParameters publicKey, ISet<string> userLabels, bool ephemeral, bool disableUpdate)
|
||||
{
|
||||
ArgUtil.NotNull(agent, nameof(agent));
|
||||
agent.Authorization = new TaskAgentAuthorization
|
||||
@@ -571,16 +528,9 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
|
||||
agent.Labels.Clear();
|
||||
|
||||
if (!noDefaultLabels)
|
||||
{
|
||||
agent.Labels.Add(new AgentLabel("self-hosted", LabelType.System));
|
||||
agent.Labels.Add(new AgentLabel(VarUtil.OS, LabelType.System));
|
||||
agent.Labels.Add(new AgentLabel(VarUtil.OSArchitecture, LabelType.System));
|
||||
}
|
||||
else if (userLabels.Count == 0)
|
||||
{
|
||||
throw new NotSupportedException("Disabling default labels via --no-default-labels without specifying --labels is not supported");
|
||||
}
|
||||
agent.Labels.Add(new AgentLabel("self-hosted", LabelType.System));
|
||||
agent.Labels.Add(new AgentLabel(VarUtil.OS, LabelType.System));
|
||||
agent.Labels.Add(new AgentLabel(VarUtil.OSArchitecture, LabelType.System));
|
||||
|
||||
foreach (var userLabel in userLabels)
|
||||
{
|
||||
@@ -590,7 +540,7 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
return agent;
|
||||
}
|
||||
|
||||
private TaskAgent CreateNewAgent(string agentName, RSAParameters publicKey, ISet<string> userLabels, bool ephemeral, bool disableUpdate, bool noDefaultLabels)
|
||||
private TaskAgent CreateNewAgent(string agentName, RSAParameters publicKey, ISet<string> userLabels, bool ephemeral, bool disableUpdate)
|
||||
{
|
||||
TaskAgent agent = new(agentName)
|
||||
{
|
||||
@@ -605,16 +555,9 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
DisableUpdate = disableUpdate
|
||||
};
|
||||
|
||||
if (!noDefaultLabels)
|
||||
{
|
||||
agent.Labels.Add(new AgentLabel("self-hosted", LabelType.System));
|
||||
agent.Labels.Add(new AgentLabel(VarUtil.OS, LabelType.System));
|
||||
agent.Labels.Add(new AgentLabel(VarUtil.OSArchitecture, LabelType.System));
|
||||
}
|
||||
else if (userLabels.Count == 0)
|
||||
{
|
||||
throw new NotSupportedException("Disabling default labels via --no-default-labels without specifying --labels is not supported");
|
||||
}
|
||||
agent.Labels.Add(new AgentLabel("self-hosted", LabelType.System));
|
||||
agent.Labels.Add(new AgentLabel(VarUtil.OS, LabelType.System));
|
||||
agent.Labels.Add(new AgentLabel(VarUtil.OSArchitecture, LabelType.System));
|
||||
|
||||
foreach (var userLabel in userLabels)
|
||||
{
|
||||
@@ -709,7 +652,7 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
{
|
||||
var response = await httpClient.PostAsync(githubApiUrl, new StringContent(string.Empty));
|
||||
responseStatus = response.StatusCode;
|
||||
var githubRequestId = _dotcomServer.GetGitHubRequestId(response.Headers);
|
||||
var githubRequestId = GetGitHubRequestId(response.Headers);
|
||||
|
||||
if (response.IsSuccessStatusCode)
|
||||
{
|
||||
@@ -772,7 +715,7 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
{
|
||||
var response = await httpClient.PostAsync(githubApiUrl, new StringContent(StringUtil.ConvertToJson(bodyObject), null, "application/json"));
|
||||
responseStatus = response.StatusCode;
|
||||
var githubRequestId = _dotcomServer.GetGitHubRequestId(response.Headers);
|
||||
var githubRequestId = GetGitHubRequestId(response.Headers);
|
||||
|
||||
if (response.IsSuccessStatusCode)
|
||||
{
|
||||
@@ -801,5 +744,14 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private string GetGitHubRequestId(HttpResponseHeaders headers)
|
||||
{
|
||||
if (headers.TryGetValues("x-github-request-id", out var headerValues))
|
||||
{
|
||||
return headerValues.FirstOrDefault();
|
||||
}
|
||||
return string.Empty;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using System;
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Runtime.Serialization;
|
||||
using GitHub.Runner.Common;
|
||||
@@ -20,8 +20,8 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
{
|
||||
public static readonly Dictionary<string, Type> CredentialTypes = new(StringComparer.OrdinalIgnoreCase)
|
||||
{
|
||||
{ Constants.Configuration.OAuth, typeof(OAuthCredential) },
|
||||
{ Constants.Configuration.OAuthAccessToken, typeof(OAuthAccessTokenCredential) },
|
||||
{ Constants.Configuration.OAuth, typeof(OAuthCredential)},
|
||||
{ Constants.Configuration.OAuthAccessToken, typeof(OAuthAccessTokenCredential)},
|
||||
};
|
||||
|
||||
public ICredentialProvider GetCredentialProvider(string credType)
|
||||
@@ -93,9 +93,6 @@ namespace GitHub.Runner.Listener.Configuration
|
||||
[DataMember(Name = "token")]
|
||||
public string Token { get; set; }
|
||||
|
||||
[DataMember(Name = "use_v2_flow")]
|
||||
public bool UseV2Flow { get; set; }
|
||||
|
||||
public VssCredentials ToVssCredentials()
|
||||
{
|
||||
ArgUtil.NotNullOrEmpty(TokenSchema, nameof(TokenSchema));
|
||||
|
||||
@@ -15,7 +15,6 @@ using GitHub.Runner.Sdk;
|
||||
using GitHub.Services.Common;
|
||||
using GitHub.Services.WebApi;
|
||||
using GitHub.Services.WebApi.Jwt;
|
||||
using Sdk.RSWebApi.Contracts;
|
||||
using Pipelines = GitHub.DistributedTask.Pipelines;
|
||||
|
||||
namespace GitHub.Runner.Listener
|
||||
@@ -373,8 +372,6 @@ namespace GitHub.Runner.Listener
|
||||
TaskCompletionSource<int> firstJobRequestRenewed = new();
|
||||
var notification = HostContext.GetService<IJobNotification>();
|
||||
|
||||
var systemConnection = message.Resources.Endpoints.SingleOrDefault(x => string.Equals(x.Name, WellKnownServiceEndpointNames.SystemVssConnection, StringComparison.OrdinalIgnoreCase));
|
||||
|
||||
// lock renew cancellation token.
|
||||
using (var lockRenewalTokenSource = new CancellationTokenSource())
|
||||
using (var workerProcessCancelTokenSource = new CancellationTokenSource())
|
||||
@@ -382,6 +379,8 @@ namespace GitHub.Runner.Listener
|
||||
long requestId = message.RequestId;
|
||||
Guid lockToken = Guid.Empty; // lockToken has never been used, keep this here of compat
|
||||
|
||||
var systemConnection = message.Resources.Endpoints.SingleOrDefault(x => string.Equals(x.Name, WellKnownServiceEndpointNames.SystemVssConnection, StringComparison.OrdinalIgnoreCase));
|
||||
|
||||
// start renew job request
|
||||
Trace.Info($"Start renew job request {requestId} for job {message.JobId}.");
|
||||
Task renewJobRequest = RenewJobRequestAsync(message, systemConnection, _poolId, requestId, lockToken, orchestrationId, firstJobRequestRenewed, lockRenewalTokenSource.Token);
|
||||
@@ -406,7 +405,7 @@ namespace GitHub.Runner.Listener
|
||||
await renewJobRequest;
|
||||
|
||||
// complete job request with result Cancelled
|
||||
await CompleteJobRequestAsync(_poolId, message, systemConnection, lockToken, TaskResult.Canceled);
|
||||
await CompleteJobRequestAsync(_poolId, message, lockToken, TaskResult.Canceled);
|
||||
return;
|
||||
}
|
||||
|
||||
@@ -545,6 +544,7 @@ namespace GitHub.Runner.Listener
|
||||
detailInfo = string.Join(Environment.NewLine, workerOutput);
|
||||
Trace.Info($"Return code {returnCode} indicate worker encounter an unhandled exception or app crash, attach worker stdout/stderr to JobRequest result.");
|
||||
|
||||
|
||||
var jobServer = await InitializeJobServerAsync(systemConnection);
|
||||
await LogWorkerProcessUnhandledException(jobServer, message, detailInfo);
|
||||
|
||||
@@ -552,7 +552,7 @@ namespace GitHub.Runner.Listener
|
||||
if (detailInfo.Contains(typeof(System.IO.IOException).ToString(), StringComparison.OrdinalIgnoreCase))
|
||||
{
|
||||
Trace.Info($"Finish job with result 'Failed' due to IOException.");
|
||||
await ForceFailJob(jobServer, message, detailInfo);
|
||||
await ForceFailJob(jobServer, message);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -567,7 +567,7 @@ namespace GitHub.Runner.Listener
|
||||
await renewJobRequest;
|
||||
|
||||
// complete job request
|
||||
await CompleteJobRequestAsync(_poolId, message, systemConnection, lockToken, result, detailInfo);
|
||||
await CompleteJobRequestAsync(_poolId, message, lockToken, result, detailInfo);
|
||||
|
||||
// print out unhandled exception happened in worker after we complete job request.
|
||||
// when we run out of disk space, report back to server has higher priority.
|
||||
@@ -664,7 +664,7 @@ namespace GitHub.Runner.Listener
|
||||
await renewJobRequest;
|
||||
|
||||
// complete job request
|
||||
await CompleteJobRequestAsync(_poolId, message, systemConnection, lockToken, resultOnAbandonOrCancel);
|
||||
await CompleteJobRequestAsync(_poolId, message, lockToken, resultOnAbandonOrCancel);
|
||||
}
|
||||
finally
|
||||
{
|
||||
@@ -1065,7 +1065,7 @@ namespace GitHub.Runner.Listener
|
||||
}
|
||||
}
|
||||
|
||||
private async Task CompleteJobRequestAsync(int poolId, Pipelines.AgentJobRequestMessage message, ServiceEndpoint systemConnection, Guid lockToken, TaskResult result, string detailInfo = null)
|
||||
private async Task CompleteJobRequestAsync(int poolId, Pipelines.AgentJobRequestMessage message, Guid lockToken, TaskResult result, string detailInfo = null)
|
||||
{
|
||||
Trace.Entering();
|
||||
|
||||
@@ -1077,23 +1077,7 @@ namespace GitHub.Runner.Listener
|
||||
|
||||
if (this._isRunServiceJob)
|
||||
{
|
||||
var runServer = await GetRunServerAsync(systemConnection);
|
||||
var unhandledExceptionIssue = new Issue() { Type = IssueType.Error, Message = detailInfo };
|
||||
var unhandledAnnotation = unhandledExceptionIssue.ToAnnotation();
|
||||
var jobAnnotations = new List<Annotation>();
|
||||
if (unhandledAnnotation.HasValue)
|
||||
{
|
||||
jobAnnotations.Add(unhandledAnnotation.Value);
|
||||
}
|
||||
try
|
||||
{
|
||||
await runServer.CompleteJobAsync(message.Plan.PlanId, message.JobId, result, outputs: null, stepResults: null, jobAnnotations: jobAnnotations, CancellationToken.None);
|
||||
}
|
||||
catch (Exception ex)
|
||||
{
|
||||
Trace.Error("Fail to raise job completion back to service.");
|
||||
Trace.Error(ex);
|
||||
}
|
||||
Trace.Verbose($"Skip FinishAgentRequest call from Listener because MessageType is {message.MessageType}");
|
||||
return;
|
||||
}
|
||||
|
||||
@@ -1133,7 +1117,7 @@ namespace GitHub.Runner.Listener
|
||||
}
|
||||
|
||||
// log an error issue to job level timeline record
|
||||
private async Task LogWorkerProcessUnhandledException(IRunnerService server, Pipelines.AgentJobRequestMessage message, string detailInfo)
|
||||
private async Task LogWorkerProcessUnhandledException(IRunnerService server, Pipelines.AgentJobRequestMessage message, string errorMessage)
|
||||
{
|
||||
if (server is IJobServer jobServer)
|
||||
{
|
||||
@@ -1145,11 +1129,34 @@ namespace GitHub.Runner.Listener
|
||||
TimelineRecord jobRecord = timeline.Records.FirstOrDefault(x => x.Id == message.JobId && x.RecordType == "Job");
|
||||
ArgUtil.NotNull(jobRecord, nameof(jobRecord));
|
||||
|
||||
var unhandledExceptionIssue = new Issue() { Type = IssueType.Error, Message = detailInfo };
|
||||
try
|
||||
{
|
||||
if (!string.IsNullOrEmpty(errorMessage) &&
|
||||
message.Variables.TryGetValue("DistributedTask.EnableRunnerIPCDebug", out var enableRunnerIPCDebug) &&
|
||||
StringUtil.ConvertToBoolean(enableRunnerIPCDebug.Value))
|
||||
{
|
||||
// the trace should be best effort and not affect any job result
|
||||
var match = _invalidJsonRegex.Match(errorMessage);
|
||||
if (match.Success &&
|
||||
match.Groups.Count == 2)
|
||||
{
|
||||
var jsonPosition = int.Parse(match.Groups[1].Value);
|
||||
var serializedJobMessage = JsonUtility.ToString(message);
|
||||
var originalJson = serializedJobMessage.Substring(jsonPosition - 10, 20);
|
||||
errorMessage = $"Runner sent Json at position '{jsonPosition}': {originalJson} ({Convert.ToBase64String(Encoding.UTF8.GetBytes(originalJson))})\n{errorMessage}";
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception ex)
|
||||
{
|
||||
Trace.Error(ex);
|
||||
errorMessage = $"Fail to check json IPC error: {ex.Message}\n{errorMessage}";
|
||||
}
|
||||
|
||||
var unhandledExceptionIssue = new Issue() { Type = IssueType.Error, Message = errorMessage };
|
||||
unhandledExceptionIssue.Data[Constants.Runner.InternalTelemetryIssueDataKey] = Constants.Runner.WorkerCrash;
|
||||
jobRecord.ErrorCount++;
|
||||
jobRecord.Issues.Add(unhandledExceptionIssue);
|
||||
|
||||
await jobServer.UpdateTimelineRecordsAsync(message.Plan.ScopeIdentifier, message.Plan.PlanType, message.Plan.PlanId, message.Timeline.Id, new TimelineRecord[] { jobRecord }, CancellationToken.None);
|
||||
}
|
||||
catch (Exception ex)
|
||||
@@ -1160,13 +1167,13 @@ namespace GitHub.Runner.Listener
|
||||
}
|
||||
else
|
||||
{
|
||||
Trace.Info("Job server does not support handling unhandled exception yet, error message: {0}", detailInfo);
|
||||
Trace.Info("Job server does not support handling unhandled exception yet, error message: {0}", errorMessage);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
// raise job completed event to fail the job.
|
||||
private async Task ForceFailJob(IRunnerService server, Pipelines.AgentJobRequestMessage message, string detailInfo)
|
||||
private async Task ForceFailJob(IRunnerService server, Pipelines.AgentJobRequestMessage message)
|
||||
{
|
||||
if (server is IJobServer jobServer)
|
||||
{
|
||||
@@ -1185,15 +1192,7 @@ namespace GitHub.Runner.Listener
|
||||
{
|
||||
try
|
||||
{
|
||||
var unhandledExceptionIssue = new Issue() { Type = IssueType.Error, Message = detailInfo };
|
||||
var unhandledAnnotation = unhandledExceptionIssue.ToAnnotation();
|
||||
var jobAnnotations = new List<Annotation>();
|
||||
if (unhandledAnnotation.HasValue)
|
||||
{
|
||||
jobAnnotations.Add(unhandledAnnotation.Value);
|
||||
}
|
||||
|
||||
await runServer.CompleteJobAsync(message.Plan.PlanId, message.JobId, TaskResult.Failed, outputs: null, stepResults: null, jobAnnotations: jobAnnotations, CancellationToken.None);
|
||||
await runServer.CompleteJobAsync(message.Plan.PlanId, message.JobId, TaskResult.Failed, outputs: null, stepResults: null, CancellationToken.None);
|
||||
}
|
||||
catch (Exception ex)
|
||||
{
|
||||
|
||||
@@ -4,7 +4,6 @@ using System.IO;
|
||||
using System.Linq;
|
||||
using System.Reflection;
|
||||
using System.Runtime.CompilerServices;
|
||||
using System.Security.Cryptography;
|
||||
using System.Text;
|
||||
using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
@@ -211,16 +210,10 @@ namespace GitHub.Runner.Listener
|
||||
foreach (var config in jitConfig)
|
||||
{
|
||||
var configFile = Path.Combine(HostContext.GetDirectory(WellKnownDirectory.Root), config.Key);
|
||||
var configContent = Convert.FromBase64String(config.Value);
|
||||
#if OS_WINDOWS
|
||||
if (configFile == HostContext.GetConfigFile(WellKnownConfigFile.RSACredentials))
|
||||
{
|
||||
configContent = ProtectedData.Protect(configContent, null, DataProtectionScope.LocalMachine);
|
||||
}
|
||||
#endif
|
||||
File.WriteAllBytes(configFile, configContent);
|
||||
var configContent = Encoding.UTF8.GetString(Convert.FromBase64String(config.Value));
|
||||
File.WriteAllText(configFile, configContent, Encoding.UTF8);
|
||||
File.SetAttributes(configFile, File.GetAttributes(configFile) | FileAttributes.Hidden);
|
||||
Trace.Info($"Saved {configContent.Length} bytes to '{configFile}'.");
|
||||
Trace.Info($"Save {configContent.Length} chars to '{configFile}'.");
|
||||
}
|
||||
}
|
||||
catch (Exception ex)
|
||||
@@ -339,26 +332,13 @@ namespace GitHub.Runner.Listener
|
||||
}
|
||||
}
|
||||
|
||||
private IMessageListener GetMesageListener(RunnerSettings settings)
|
||||
{
|
||||
if (settings.UseV2Flow)
|
||||
{
|
||||
Trace.Info($"Using BrokerMessageListener");
|
||||
var brokerListener = new BrokerMessageListener();
|
||||
brokerListener.Initialize(HostContext);
|
||||
return brokerListener;
|
||||
}
|
||||
|
||||
return HostContext.GetService<IMessageListener>();
|
||||
}
|
||||
|
||||
//create worker manager, create message listener and start listening to the queue
|
||||
private async Task<int> RunAsync(RunnerSettings settings, bool runOnce = false)
|
||||
{
|
||||
try
|
||||
{
|
||||
Trace.Info(nameof(RunAsync));
|
||||
_listener = GetMesageListener(settings);
|
||||
_listener = HostContext.GetService<IMessageListener>();
|
||||
if (!await _listener.CreateSessionAsync(HostContext.RunnerShutdownToken))
|
||||
{
|
||||
return Constants.Runner.ReturnCode.TerminatedError;
|
||||
@@ -549,17 +529,7 @@ namespace GitHub.Runner.Listener
|
||||
{
|
||||
var runServer = HostContext.CreateService<IRunServer>();
|
||||
await runServer.ConnectAsync(new Uri(messageRef.RunServiceUrl), creds);
|
||||
try
|
||||
{
|
||||
jobRequestMessage =
|
||||
await runServer.GetJobMessageAsync(messageRef.RunnerRequestId,
|
||||
messageQueueLoopTokenSource.Token);
|
||||
}
|
||||
catch (TaskOrchestrationJobAlreadyAcquiredException)
|
||||
{
|
||||
Trace.Info("Job is already acquired, skip this message.");
|
||||
continue;
|
||||
}
|
||||
jobRequestMessage = await runServer.GetJobMessageAsync(messageRef.RunnerRequestId, messageQueueLoopTokenSource.Token);
|
||||
}
|
||||
|
||||
jobDispatcher.Run(jobRequestMessage, runOnce);
|
||||
@@ -683,8 +653,7 @@ Config Options:
|
||||
--token string Registration token. Required if unattended
|
||||
--name string Name of the runner to configure (default {Environment.MachineName ?? "myrunner"})
|
||||
--runnergroup string Name of the runner group to add this runner to (defaults to the default runner group)
|
||||
--labels string Custom labels that will be added to the runner. This option is mandatory if --no-default-labels is used.
|
||||
--no-default-labels Disables adding the default labels: 'self-hosted,{Constants.Runner.Platform},{Constants.Runner.PlatformArchitecture}'
|
||||
--labels string Extra labels in addition to the default: 'self-hosted,{Constants.Runner.Platform},{Constants.Runner.PlatformArchitecture}'
|
||||
--local Removes the runner config files from your local machine. Used as an option to the remove command
|
||||
--work string Relative runner work directory (default {Constants.Path.WorkDirectory})
|
||||
--replace Replace any existing runner with the same name (default false)
|
||||
|
||||
@@ -40,19 +40,10 @@ namespace GitHub.Runner.Sdk
|
||||
File.WriteAllText(path, StringUtil.ConvertToJson(obj), Encoding.UTF8);
|
||||
}
|
||||
|
||||
public static T LoadObject<T>(string path, bool required = false)
|
||||
public static T LoadObject<T>(string path)
|
||||
{
|
||||
string json = File.ReadAllText(path, Encoding.UTF8);
|
||||
if (required && string.IsNullOrEmpty(json))
|
||||
{
|
||||
throw new ArgumentNullException($"File {path} is empty");
|
||||
}
|
||||
T result = StringUtil.ConvertFromJson<T>(json);
|
||||
if (required && result == null)
|
||||
{
|
||||
throw new ArgumentException("Converting json to object resulted in a null value");
|
||||
}
|
||||
return result;
|
||||
return StringUtil.ConvertFromJson<T>(json);
|
||||
}
|
||||
|
||||
public static string GetSha256Hash(string path)
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using System;
|
||||
using System;
|
||||
|
||||
namespace GitHub.Runner.Sdk
|
||||
{
|
||||
@@ -15,7 +15,6 @@ namespace GitHub.Runner.Sdk
|
||||
string.Equals(gitHubUrl.Host, "github.com", StringComparison.OrdinalIgnoreCase) ||
|
||||
string.Equals(gitHubUrl.Host, "www.github.com", StringComparison.OrdinalIgnoreCase) ||
|
||||
string.Equals(gitHubUrl.Host, "github.localhost", StringComparison.OrdinalIgnoreCase) ||
|
||||
gitHubUrl.Host.EndsWith(".ghe.localhost", StringComparison.OrdinalIgnoreCase) ||
|
||||
gitHubUrl.Host.EndsWith(".ghe.com", StringComparison.OrdinalIgnoreCase);
|
||||
}
|
||||
|
||||
|
||||
@@ -60,7 +60,7 @@ namespace GitHub.Runner.Sdk
|
||||
trace?.Verbose(ex.ToString());
|
||||
}
|
||||
|
||||
if (matches != null && matches.Length > 0 && IsPathValid(matches.First(), trace))
|
||||
if (matches != null && matches.Length > 0)
|
||||
{
|
||||
trace?.Info($"Location: '{matches.First()}'");
|
||||
return matches.First();
|
||||
@@ -86,7 +86,7 @@ namespace GitHub.Runner.Sdk
|
||||
for (int i = 0; i < pathExtSegments.Length; i++)
|
||||
{
|
||||
string fullPath = Path.Combine(pathSegment, $"{command}{pathExtSegments[i]}");
|
||||
if (matches.Any(p => p.Equals(fullPath, StringComparison.OrdinalIgnoreCase)) && IsPathValid(fullPath, trace))
|
||||
if (matches.Any(p => p.Equals(fullPath, StringComparison.OrdinalIgnoreCase)))
|
||||
{
|
||||
trace?.Info($"Location: '{fullPath}'");
|
||||
return fullPath;
|
||||
@@ -105,7 +105,7 @@ namespace GitHub.Runner.Sdk
|
||||
trace?.Verbose(ex.ToString());
|
||||
}
|
||||
|
||||
if (matches != null && matches.Length > 0 && IsPathValid(matches.First(), trace))
|
||||
if (matches != null && matches.Length > 0)
|
||||
{
|
||||
trace?.Info($"Location: '{matches.First()}'");
|
||||
return matches.First();
|
||||
@@ -128,15 +128,5 @@ namespace GitHub.Runner.Sdk
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
// checks if the file is a symlink and if the symlink`s target exists.
|
||||
private static bool IsPathValid(string path, ITraceWriter trace = null)
|
||||
{
|
||||
var fileInfo = new FileInfo(path);
|
||||
var linkTargetFullPath = fileInfo.Directory?.FullName + Path.DirectorySeparatorChar + fileInfo.LinkTarget;
|
||||
if(fileInfo.LinkTarget == null || File.Exists(linkTargetFullPath) || File.Exists(fileInfo.LinkTarget)) return true;
|
||||
trace?.Info($"the target '{fileInfo.LinkTarget}' of the symbolic link '{path}', does not exist");
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using GitHub.DistributedTask.Pipelines.ContextData;
|
||||
using GitHub.DistributedTask.Pipelines.ContextData;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using GitHub.Runner.Worker.Container;
|
||||
using System;
|
||||
@@ -276,7 +276,7 @@ namespace GitHub.Runner.Worker
|
||||
Message = $"Can't update {blocked} environment variable using ::set-env:: command."
|
||||
};
|
||||
issue.Data[Constants.Runner.InternalTelemetryIssueDataKey] = $"{Constants.Runner.UnsupportedCommand}_{envName}";
|
||||
context.AddIssue(issue, ExecutionContextLogOptions.Default);
|
||||
context.AddIssue(issue);
|
||||
|
||||
return;
|
||||
}
|
||||
@@ -315,7 +315,7 @@ namespace GitHub.Runner.Worker
|
||||
Message = String.Format(Constants.Runner.UnsupportedCommandMessage, this.Command)
|
||||
};
|
||||
issue.Data[Constants.Runner.InternalTelemetryIssueDataKey] = Constants.Runner.UnsupportedCommand;
|
||||
context.AddIssue(issue, ExecutionContextLogOptions.Default);
|
||||
context.AddIssue(issue);
|
||||
}
|
||||
|
||||
if (!command.Properties.TryGetValue(SetOutputCommandProperties.Name, out string outputName) || string.IsNullOrEmpty(outputName))
|
||||
@@ -350,7 +350,7 @@ namespace GitHub.Runner.Worker
|
||||
Message = String.Format(Constants.Runner.UnsupportedCommandMessage, this.Command)
|
||||
};
|
||||
issue.Data[Constants.Runner.InternalTelemetryIssueDataKey] = Constants.Runner.UnsupportedCommand;
|
||||
context.AddIssue(issue, ExecutionContextLogOptions.Default);
|
||||
context.AddIssue(issue);
|
||||
}
|
||||
|
||||
if (!command.Properties.TryGetValue(SaveStateCommandProperties.Name, out string stateName) || string.IsNullOrEmpty(stateName))
|
||||
@@ -666,7 +666,7 @@ namespace GitHub.Runner.Worker
|
||||
}
|
||||
}
|
||||
|
||||
context.AddIssue(issue, ExecutionContextLogOptions.Default);
|
||||
context.AddIssue(issue);
|
||||
}
|
||||
|
||||
public static void ValidateLinesAndColumns(ActionCommand command, IExecutionContext context)
|
||||
|
||||
@@ -11,14 +11,12 @@ using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
using GitHub.DistributedTask.ObjectTemplating.Tokens;
|
||||
using GitHub.Runner.Common;
|
||||
using GitHub.Runner.Common.Util;
|
||||
using GitHub.Runner.Sdk;
|
||||
using GitHub.Runner.Worker.Container;
|
||||
using GitHub.Services.Common;
|
||||
using WebApi = GitHub.DistributedTask.WebApi;
|
||||
using Pipelines = GitHub.DistributedTask.Pipelines;
|
||||
using PipelineTemplateConstants = GitHub.DistributedTask.Pipelines.ObjectTemplating.PipelineTemplateConstants;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
|
||||
namespace GitHub.Runner.Worker
|
||||
{
|
||||
@@ -102,19 +100,7 @@ namespace GitHub.Runner.Worker
|
||||
}
|
||||
IEnumerable<Pipelines.ActionStep> actions = steps.OfType<Pipelines.ActionStep>();
|
||||
executionContext.Output("Prepare all required actions");
|
||||
PrepareActionsState result = new PrepareActionsState();
|
||||
try
|
||||
{
|
||||
result = await PrepareActionsRecursiveAsync(executionContext, state, actions, depth, rootStepId);
|
||||
}
|
||||
catch (FailedToResolveActionDownloadInfoException ex)
|
||||
{
|
||||
// Log the error and fail the PrepareActionsAsync Initialization.
|
||||
Trace.Error($"Caught exception from PrepareActionsAsync Initialization: {ex}");
|
||||
executionContext.InfrastructureError(ex.Message);
|
||||
executionContext.Result = TaskResult.Failed;
|
||||
throw;
|
||||
}
|
||||
var result = await PrepareActionsRecursiveAsync(executionContext, state, actions, depth, rootStepId);
|
||||
if (!FeatureManager.IsContainerHooksEnabled(executionContext.Global.Variables))
|
||||
{
|
||||
if (state.ImagesToPull.Count > 0)
|
||||
@@ -662,21 +648,13 @@ namespace GitHub.Runner.Worker
|
||||
}
|
||||
|
||||
// Resolve download info
|
||||
var launchServer = HostContext.GetService<ILaunchServer>();
|
||||
var jobServer = HostContext.GetService<IJobServer>();
|
||||
var actionDownloadInfos = default(WebApi.ActionDownloadInfoCollection);
|
||||
for (var attempt = 1; attempt <= 3; attempt++)
|
||||
{
|
||||
try
|
||||
{
|
||||
if (MessageUtil.IsRunServiceJob(executionContext.Global.Variables.Get(Constants.Variables.System.JobRequestType)))
|
||||
{
|
||||
actionDownloadInfos = await launchServer.ResolveActionsDownloadInfoAsync(executionContext.Global.Plan.PlanId, executionContext.Root.Id, new WebApi.ActionReferenceList { Actions = actionReferences }, executionContext.CancellationToken);
|
||||
}
|
||||
else
|
||||
{
|
||||
actionDownloadInfos = await jobServer.ResolveActionDownloadInfoAsync(executionContext.Global.Plan.ScopeIdentifier, executionContext.Global.Plan.PlanType, executionContext.Global.Plan.PlanId, executionContext.Root.Id, new WebApi.ActionReferenceList { Actions = actionReferences }, executionContext.CancellationToken);
|
||||
}
|
||||
actionDownloadInfos = await jobServer.ResolveActionDownloadInfoAsync(executionContext.Global.Plan.ScopeIdentifier, executionContext.Global.Plan.PlanType, executionContext.Global.Plan.PlanId, executionContext.Root.Id, new WebApi.ActionReferenceList { Actions = actionReferences }, executionContext.CancellationToken);
|
||||
break;
|
||||
}
|
||||
catch (Exception ex) when (!executionContext.CancellationToken.IsCancellationRequested) // Do not retry if the run is cancelled.
|
||||
|
||||
@@ -18,22 +18,15 @@ using GitHub.Runner.Sdk;
|
||||
using GitHub.Runner.Worker.Container;
|
||||
using GitHub.Runner.Worker.Handlers;
|
||||
using Newtonsoft.Json;
|
||||
using Sdk.RSWebApi.Contracts;
|
||||
using ObjectTemplating = GitHub.DistributedTask.ObjectTemplating;
|
||||
using Pipelines = GitHub.DistributedTask.Pipelines;
|
||||
|
||||
namespace GitHub.Runner.Worker
|
||||
{
|
||||
public static class ExecutionContextType
|
||||
public class ExecutionContextType
|
||||
{
|
||||
public const string Job = "Job";
|
||||
public const string Task = "Task";
|
||||
}
|
||||
|
||||
public record ExecutionContextLogOptions(bool WriteToLog, string LogMessageOverride)
|
||||
{
|
||||
public static readonly ExecutionContextLogOptions None = new(false, null);
|
||||
public static readonly ExecutionContextLogOptions Default = new(true, null);
|
||||
public static string Job = "Job";
|
||||
public static string Task = "Task";
|
||||
}
|
||||
|
||||
[ServiceLocator(Default = typeof(ExecutionContext))]
|
||||
@@ -99,7 +92,7 @@ namespace GitHub.Runner.Worker
|
||||
void SetGitHubContext(string name, string value);
|
||||
void SetOutput(string name, string value, out string reference);
|
||||
void SetTimeout(TimeSpan? timeout);
|
||||
void AddIssue(Issue issue, ExecutionContextLogOptions logOptions);
|
||||
void AddIssue(Issue issue, string message = null);
|
||||
void Progress(int percentage, string currentOperation = null);
|
||||
void UpdateDetailTimelineRecord(TimelineRecord record);
|
||||
|
||||
@@ -125,7 +118,7 @@ namespace GitHub.Runner.Worker
|
||||
|
||||
public sealed class ExecutionContext : RunnerService, IExecutionContext
|
||||
{
|
||||
private const int _maxCountPerIssueType = 10;
|
||||
private const int _maxIssueCount = 10;
|
||||
private const int _throttlingDelayReportThreshold = 10 * 1000; // Don't report throttling with less than 10 seconds delay
|
||||
private const int _maxIssueMessageLength = 4096; // Don't send issue with huge message since we can't forward them from actions to check annotation.
|
||||
private const int _maxIssueCountInTelemetry = 3; // Only send the first 3 issues to telemetry
|
||||
@@ -133,10 +126,8 @@ namespace GitHub.Runner.Worker
|
||||
|
||||
private readonly TimelineRecord _record = new();
|
||||
private readonly Dictionary<Guid, TimelineRecord> _detailRecords = new();
|
||||
private readonly List<Issue> _embeddedIssueCollector;
|
||||
private readonly object _loggerLock = new();
|
||||
private readonly object _matchersLock = new();
|
||||
private readonly ExecutionContext _parentExecutionContext;
|
||||
|
||||
private event OnMatcherChanged _onMatcherChanged;
|
||||
|
||||
@@ -144,6 +135,7 @@ namespace GitHub.Runner.Worker
|
||||
|
||||
private IPagingLogger _logger;
|
||||
private IJobServerQueue _jobServerQueue;
|
||||
private ExecutionContext _parentExecutionContext;
|
||||
|
||||
private Guid _mainTimelineId;
|
||||
private Guid _detailTimelineId;
|
||||
@@ -157,29 +149,6 @@ namespace GitHub.Runner.Worker
|
||||
private long _totalThrottlingDelayInMilliseconds = 0;
|
||||
private bool _stepTelemetryPublished = false;
|
||||
|
||||
public ExecutionContext()
|
||||
: this(parent: null, embedded: false)
|
||||
{
|
||||
}
|
||||
|
||||
private ExecutionContext(ExecutionContext parent, bool embedded)
|
||||
{
|
||||
if (embedded)
|
||||
{
|
||||
ArgUtil.NotNull(parent, nameof(parent));
|
||||
}
|
||||
|
||||
_parentExecutionContext = parent;
|
||||
this.IsEmbedded = embedded;
|
||||
this.StepTelemetry = new ActionsStepTelemetry
|
||||
{
|
||||
IsEmbedded = embedded
|
||||
};
|
||||
|
||||
//Embedded Execution Contexts pseudo-inherit their parent's embeddedIssueCollector.
|
||||
_embeddedIssueCollector = embedded ? parent._embeddedIssueCollector : new();
|
||||
}
|
||||
|
||||
public Guid Id => _record.Id;
|
||||
public Guid EmbeddedId { get; private set; }
|
||||
public string ScopeName { get; private set; }
|
||||
@@ -192,7 +161,7 @@ namespace GitHub.Runner.Worker
|
||||
public Dictionary<string, VariableValue> JobOutputs { get; private set; }
|
||||
|
||||
public ActionsEnvironmentReference ActionsEnvironment { get; private set; }
|
||||
public ActionsStepTelemetry StepTelemetry { get; private init; }
|
||||
public ActionsStepTelemetry StepTelemetry { get; } = new ActionsStepTelemetry();
|
||||
public DictionaryContextData ExpressionValues { get; } = new DictionaryContextData();
|
||||
public IList<IFunctionInfo> ExpressionFunctions { get; } = new List<IFunctionInfo>();
|
||||
|
||||
@@ -217,7 +186,7 @@ namespace GitHub.Runner.Worker
|
||||
|
||||
// An embedded execution context shares the same record ID, record name, and logger
|
||||
// as its enclosing execution context.
|
||||
public bool IsEmbedded { get; private init; }
|
||||
public bool IsEmbedded { get; private set; }
|
||||
|
||||
public TaskResult? Result
|
||||
{
|
||||
@@ -352,7 +321,7 @@ namespace GitHub.Runner.Worker
|
||||
{
|
||||
Trace.Entering();
|
||||
|
||||
var child = new ExecutionContext(this, isEmbedded);
|
||||
var child = new ExecutionContext();
|
||||
child.Initialize(HostContext);
|
||||
child.Global = Global;
|
||||
child.ScopeName = scopeName;
|
||||
@@ -377,6 +346,7 @@ namespace GitHub.Runner.Worker
|
||||
child.ExpressionFunctions.Add(item);
|
||||
}
|
||||
child._cancellationTokenSource = cancellationTokenSource ?? new CancellationTokenSource();
|
||||
child._parentExecutionContext = this;
|
||||
child.EchoOnActionCommand = EchoOnActionCommand;
|
||||
|
||||
if (recordOrder != null)
|
||||
@@ -397,9 +367,11 @@ namespace GitHub.Runner.Worker
|
||||
child._logger.Setup(_mainTimelineId, recordId);
|
||||
}
|
||||
|
||||
child.IsEmbedded = isEmbedded;
|
||||
child.StepTelemetry.StepId = recordId;
|
||||
child.StepTelemetry.Stage = stage.ToString();
|
||||
child.StepTelemetry.StepContextName = child.GetFullyQualifiedContextName();
|
||||
child.StepTelemetry.IsEmbedded = isEmbedded;
|
||||
child.StepTelemetry.StepContextName = child.GetFullyQualifiedContextName(); ;
|
||||
|
||||
return child;
|
||||
}
|
||||
@@ -441,24 +413,13 @@ namespace GitHub.Runner.Worker
|
||||
this.Warning($"The job has experienced {TimeSpan.FromMilliseconds(_totalThrottlingDelayInMilliseconds).TotalSeconds} seconds total delay caused by server throttling.");
|
||||
}
|
||||
|
||||
DateTime now = DateTime.UtcNow;
|
||||
_record.CurrentOperation = currentOperation ?? _record.CurrentOperation;
|
||||
_record.ResultCode = resultCode ?? _record.ResultCode;
|
||||
_record.FinishTime = now;
|
||||
_record.FinishTime = DateTime.UtcNow;
|
||||
_record.PercentComplete = 100;
|
||||
_record.Result = _record.Result ?? TaskResult.Succeeded;
|
||||
_record.State = TimelineRecordState.Completed;
|
||||
|
||||
// Before our main timeline's final QueueTimelineRecordUpdate,
|
||||
// inject any issues collected by embedded ExecutionContexts.
|
||||
if (!this.IsEmbedded)
|
||||
{
|
||||
foreach (var issue in _embeddedIssueCollector)
|
||||
{
|
||||
AddIssue(issue, ExecutionContextLogOptions.None);
|
||||
}
|
||||
}
|
||||
|
||||
_jobServerQueue.QueueTimelineRecordUpdate(_mainTimelineId, _record);
|
||||
|
||||
// complete all detail timeline records.
|
||||
@@ -466,7 +427,7 @@ namespace GitHub.Runner.Worker
|
||||
{
|
||||
foreach (var record in _detailRecords)
|
||||
{
|
||||
record.Value.FinishTime = record.Value.FinishTime ?? now;
|
||||
record.Value.FinishTime = record.Value.FinishTime ?? DateTime.UtcNow;
|
||||
record.Value.PercentComplete = record.Value.PercentComplete ?? 100;
|
||||
record.Value.Result = record.Value.Result ?? TaskResult.Succeeded;
|
||||
record.Value.State = TimelineRecordState.Completed;
|
||||
@@ -477,26 +438,14 @@ namespace GitHub.Runner.Worker
|
||||
|
||||
PublishStepTelemetry();
|
||||
|
||||
var stepResult = new StepResult
|
||||
{
|
||||
ExternalID = _record.Id,
|
||||
Conclusion = _record.Result ?? TaskResult.Succeeded,
|
||||
Status = _record.State,
|
||||
Number = _record.Order,
|
||||
Name = _record.Name,
|
||||
StartedAt = _record.StartTime,
|
||||
CompletedAt = _record.FinishTime,
|
||||
Annotations = new List<Annotation>()
|
||||
};
|
||||
|
||||
_record.Issues?.ForEach(issue =>
|
||||
{
|
||||
var annotation = issue.ToAnnotation();
|
||||
if (annotation != null)
|
||||
{
|
||||
stepResult.Annotations.Add(annotation.Value);
|
||||
}
|
||||
});
|
||||
var stepResult = new StepResult();
|
||||
stepResult.ExternalID = _record.Id;
|
||||
stepResult.Conclusion = _record.Result ?? TaskResult.Succeeded;
|
||||
stepResult.Status = _record.State;
|
||||
stepResult.Number = _record.Order;
|
||||
stepResult.Name = _record.Name;
|
||||
stepResult.StartedAt = _record.StartTime;
|
||||
stepResult.CompletedAt = _record.FinishTime;
|
||||
|
||||
Global.StepsResult.Add(stepResult);
|
||||
|
||||
@@ -610,10 +559,14 @@ namespace GitHub.Runner.Worker
|
||||
}
|
||||
|
||||
// This is not thread safe, the caller need to take lock before calling issue()
|
||||
public void AddIssue(Issue issue, ExecutionContextLogOptions logOptions)
|
||||
public void AddIssue(Issue issue, string logMessage = null)
|
||||
{
|
||||
ArgUtil.NotNull(issue, nameof(issue));
|
||||
ArgUtil.NotNull(logOptions, nameof(logOptions));
|
||||
|
||||
if (string.IsNullOrEmpty(logMessage))
|
||||
{
|
||||
logMessage = issue.Message;
|
||||
}
|
||||
|
||||
issue.Message = HostContext.SecretMasker.MaskSecrets(issue.Message);
|
||||
if (issue.Message.Length > _maxIssueMessageLength)
|
||||
@@ -628,64 +581,53 @@ namespace GitHub.Runner.Worker
|
||||
issue.Data["stepNumber"] = _record.Order.ToString();
|
||||
}
|
||||
|
||||
string wellKnownTag = null;
|
||||
Int32 previousCountForIssueType = 0;
|
||||
Action incrementIssueTypeCount = NoOp;
|
||||
switch (issue.Type)
|
||||
if (issue.Type == IssueType.Error)
|
||||
{
|
||||
case IssueType.Error:
|
||||
wellKnownTag = WellKnownTags.Error;
|
||||
previousCountForIssueType = _record.ErrorCount;
|
||||
incrementIssueTypeCount = () => { _record.ErrorCount++; };
|
||||
break;
|
||||
case IssueType.Warning:
|
||||
wellKnownTag = WellKnownTags.Warning;
|
||||
previousCountForIssueType = _record.WarningCount;
|
||||
incrementIssueTypeCount = () => { _record.WarningCount++; };
|
||||
break;
|
||||
case IssueType.Notice:
|
||||
wellKnownTag = WellKnownTags.Notice;
|
||||
previousCountForIssueType = _record.NoticeCount;
|
||||
incrementIssueTypeCount = () => { _record.NoticeCount++; };
|
||||
break;
|
||||
}
|
||||
|
||||
if (!string.IsNullOrEmpty(wellKnownTag))
|
||||
{
|
||||
if (!this.IsEmbedded && previousCountForIssueType < _maxCountPerIssueType)
|
||||
if (!string.IsNullOrEmpty(logMessage))
|
||||
{
|
||||
long logLineNumber = Write(WellKnownTags.Error, logMessage);
|
||||
issue.Data["logFileLineNumber"] = logLineNumber.ToString();
|
||||
}
|
||||
|
||||
if (_record.ErrorCount < _maxIssueCount)
|
||||
{
|
||||
incrementIssueTypeCount();
|
||||
_record.Issues.Add(issue);
|
||||
}
|
||||
|
||||
if (logOptions.WriteToLog)
|
||||
_record.ErrorCount++;
|
||||
}
|
||||
else if (issue.Type == IssueType.Warning)
|
||||
{
|
||||
if (!string.IsNullOrEmpty(logMessage))
|
||||
{
|
||||
string logMessage = issue.Message;
|
||||
if (!string.IsNullOrEmpty(logOptions.LogMessageOverride))
|
||||
{
|
||||
logMessage = logOptions.LogMessageOverride;
|
||||
}
|
||||
|
||||
if (!string.IsNullOrEmpty(logMessage))
|
||||
{
|
||||
// Note that ::Write() has its own secret-masking logic.
|
||||
long logLineNumber = Write(wellKnownTag, logMessage);
|
||||
issue.Data["logFileLineNumber"] = logLineNumber.ToString();
|
||||
}
|
||||
long logLineNumber = Write(WellKnownTags.Warning, logMessage);
|
||||
issue.Data["logFileLineNumber"] = logLineNumber.ToString();
|
||||
}
|
||||
|
||||
if (_record.WarningCount < _maxIssueCount)
|
||||
{
|
||||
_record.Issues.Add(issue);
|
||||
}
|
||||
|
||||
_record.WarningCount++;
|
||||
}
|
||||
else if (issue.Type == IssueType.Notice)
|
||||
{
|
||||
if (!string.IsNullOrEmpty(logMessage))
|
||||
{
|
||||
long logLineNumber = Write(WellKnownTags.Notice, logMessage);
|
||||
issue.Data["logFileLineNumber"] = logLineNumber.ToString();
|
||||
}
|
||||
|
||||
if (_record.NoticeCount < _maxIssueCount)
|
||||
{
|
||||
_record.Issues.Add(issue);
|
||||
}
|
||||
|
||||
_record.NoticeCount++;
|
||||
}
|
||||
|
||||
// Embedded ExecutionContexts (a.k.a. Composite actions) should never upload a timeline record to the server.
|
||||
// Instead, we store processed issues on a shared (psuedo-inherited) list (belonging to the closest
|
||||
// non-embedded ancestor ExecutionContext) so that they can be processed when that ancestor completes.
|
||||
if (this.IsEmbedded)
|
||||
{
|
||||
_embeddedIssueCollector.Add(issue);
|
||||
}
|
||||
else
|
||||
{
|
||||
_jobServerQueue.QueueTimelineRecordUpdate(_mainTimelineId, _record);
|
||||
}
|
||||
_jobServerQueue.QueueTimelineRecordUpdate(_mainTimelineId, _record);
|
||||
}
|
||||
|
||||
public void UpdateDetailTimelineRecord(TimelineRecord record)
|
||||
@@ -783,9 +725,6 @@ namespace GitHub.Runner.Worker
|
||||
// Steps results for entire job
|
||||
Global.StepsResult = new List<StepResult>();
|
||||
|
||||
// Job level annotations
|
||||
Global.JobAnnotations = new List<Annotation>();
|
||||
|
||||
// Job Outputs
|
||||
JobOutputs = new Dictionary<string, VariableValue>(StringComparer.OrdinalIgnoreCase);
|
||||
|
||||
@@ -802,9 +741,6 @@ namespace GitHub.Runner.Worker
|
||||
// File table
|
||||
Global.FileTable = new List<String>(message.FileTable ?? new string[0]);
|
||||
|
||||
// What type of job request is running (i.e. Run Service vs. pipelines)
|
||||
Global.Variables.Set(Constants.Variables.System.JobRequestType, message.MessageType);
|
||||
|
||||
// Expression values
|
||||
if (message.ContextData?.Count > 0)
|
||||
{
|
||||
@@ -1059,7 +995,8 @@ namespace GitHub.Runner.Worker
|
||||
StepTelemetry.FinishTime = _record.FinishTime;
|
||||
}
|
||||
|
||||
if (!IsEmbedded)
|
||||
if (!IsEmbedded &&
|
||||
_record.Issues.Count > 0)
|
||||
{
|
||||
foreach (var issue in _record.Issues)
|
||||
{
|
||||
@@ -1225,11 +1162,6 @@ namespace GitHub.Runner.Worker
|
||||
|
||||
UpdateGlobalStepsContext();
|
||||
}
|
||||
|
||||
private static void NoOp()
|
||||
{
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// The Error/Warning/etc methods are created as extension methods to simplify unit testing.
|
||||
@@ -1254,22 +1186,19 @@ namespace GitHub.Runner.Worker
|
||||
// Do not add a format string overload. See comment on ExecutionContext.Write().
|
||||
public static void Error(this IExecutionContext context, string message)
|
||||
{
|
||||
var issue = new Issue() { Type = IssueType.Error, Message = message };
|
||||
context.AddIssue(issue, ExecutionContextLogOptions.Default);
|
||||
context.AddIssue(new Issue() { Type = IssueType.Error, Message = message });
|
||||
}
|
||||
|
||||
// Do not add a format string overload. See comment on ExecutionContext.Write().
|
||||
public static void InfrastructureError(this IExecutionContext context, string message)
|
||||
{
|
||||
var issue = new Issue() { Type = IssueType.Error, Message = message, IsInfrastructureIssue = true };
|
||||
context.AddIssue(issue, ExecutionContextLogOptions.Default);
|
||||
context.AddIssue(new Issue() { Type = IssueType.Error, Message = message, IsInfrastructureIssue = true });
|
||||
}
|
||||
|
||||
// Do not add a format string overload. See comment on ExecutionContext.Write().
|
||||
public static void Warning(this IExecutionContext context, string message)
|
||||
{
|
||||
var issue = new Issue() { Type = IssueType.Warning, Message = message };
|
||||
context.AddIssue(issue, ExecutionContextLogOptions.Default);
|
||||
context.AddIssue(new Issue() { Type = IssueType.Warning, Message = message });
|
||||
}
|
||||
|
||||
// Do not add a format string overload. See comment on ExecutionContext.Write().
|
||||
|
||||
@@ -1,11 +1,10 @@
|
||||
using System;
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using GitHub.Actions.RunService.WebApi;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using GitHub.Runner.Common.Util;
|
||||
using GitHub.Runner.Worker.Container;
|
||||
using Newtonsoft.Json.Linq;
|
||||
using Sdk.RSWebApi.Contracts;
|
||||
|
||||
namespace GitHub.Runner.Worker
|
||||
{
|
||||
@@ -19,7 +18,6 @@ namespace GitHub.Runner.Worker
|
||||
public IDictionary<String, IDictionary<String, String>> JobDefaults { get; set; }
|
||||
public List<ActionsStepTelemetry> StepsTelemetry { get; set; }
|
||||
public List<StepResult> StepsResult { get; set; }
|
||||
public List<Annotation> JobAnnotations { get; set; }
|
||||
public List<JobTelemetry> JobTelemetry { get; set; }
|
||||
public TaskOrchestrationPlanReference Plan { get; set; }
|
||||
public List<string> PrependPath { get; set; }
|
||||
|
||||
@@ -294,7 +294,7 @@ namespace GitHub.Runner.Worker.Handlers
|
||||
// Evaluation error
|
||||
Trace.Info("Caught exception from expression for embedded step.env");
|
||||
step.ExecutionContext.Error(ex);
|
||||
SetStepConclusion(step, TaskResult.Failed);
|
||||
step.ExecutionContext.Complete(TaskResult.Failed);
|
||||
}
|
||||
|
||||
// Register Callback
|
||||
|
||||
@@ -143,8 +143,7 @@ namespace GitHub.Runner.Worker.Handlers
|
||||
if (issue != null)
|
||||
{
|
||||
// Log issue
|
||||
var logOptions = new ExecutionContextLogOptions(true, stripped);
|
||||
_executionContext.AddIssue(issue, logOptions);
|
||||
_executionContext.AddIssue(issue, stripped);
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
@@ -431,6 +431,14 @@ namespace GitHub.Runner.Worker
|
||||
context.Result = TaskResult.Canceled;
|
||||
throw;
|
||||
}
|
||||
catch (FailedToResolveActionDownloadInfoException ex)
|
||||
{
|
||||
// Log the error and fail the JobExtension Initialization.
|
||||
Trace.Error($"Caught exception from JobExtenion Initialization: {ex}");
|
||||
context.InfrastructureError(ex.Message);
|
||||
context.Result = TaskResult.Failed;
|
||||
throw;
|
||||
}
|
||||
catch (Exception ex)
|
||||
{
|
||||
// Log the error and fail the JobExtension Initialization.
|
||||
@@ -675,7 +683,7 @@ namespace GitHub.Runner.Worker
|
||||
{
|
||||
var issue = new Issue() { Type = IssueType.Warning, Message = $"You are running out of disk space. The runner will stop working when the machine runs out of disk space. Free space left: {freeSpaceInMB} MB" };
|
||||
issue.Data[Constants.Runner.InternalTelemetryIssueDataKey] = Constants.Runner.LowDiskSpace;
|
||||
context.AddIssue(issue, ExecutionContextLogOptions.Default);
|
||||
context.AddIssue(issue);
|
||||
return;
|
||||
}
|
||||
|
||||
|
||||
@@ -3,7 +3,6 @@ using System.Collections.Generic;
|
||||
using System.IO;
|
||||
using System.Linq;
|
||||
using System.Net.Http;
|
||||
using System.Net.Http.Headers;
|
||||
using System.Text;
|
||||
using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
@@ -43,13 +42,6 @@ namespace GitHub.Runner.Worker
|
||||
DateTime jobStartTimeUtc = DateTime.UtcNow;
|
||||
IRunnerService server = null;
|
||||
|
||||
// add orchestration id to useragent for better correlation.
|
||||
if (message.Variables.TryGetValue(Constants.Variables.System.OrchestrationId, out VariableValue orchestrationId) &&
|
||||
!string.IsNullOrEmpty(orchestrationId.Value))
|
||||
{
|
||||
HostContext.UserAgents.Add(new ProductInfoHeaderValue("OrchestrationId", orchestrationId.Value));
|
||||
}
|
||||
|
||||
ServiceEndpoint systemConnection = message.Resources.Endpoints.Single(x => string.Equals(x.Name, WellKnownServiceEndpointNames.SystemVssConnection, StringComparison.OrdinalIgnoreCase));
|
||||
if (MessageUtil.IsRunServiceJob(message.MessageType))
|
||||
{
|
||||
@@ -57,21 +49,6 @@ namespace GitHub.Runner.Worker
|
||||
VssCredentials jobServerCredential = VssUtil.GetVssCredential(systemConnection);
|
||||
await runServer.ConnectAsync(systemConnection.Url, jobServerCredential);
|
||||
server = runServer;
|
||||
|
||||
message.Variables.TryGetValue("system.github.launch_endpoint", out VariableValue launchEndpointVariable);
|
||||
var launchReceiverEndpoint = launchEndpointVariable?.Value;
|
||||
|
||||
if (systemConnection?.Authorization != null &&
|
||||
systemConnection.Authorization.Parameters.TryGetValue("AccessToken", out var accessToken) &&
|
||||
!string.IsNullOrEmpty(accessToken) &&
|
||||
!string.IsNullOrEmpty(launchReceiverEndpoint))
|
||||
{
|
||||
Trace.Info("Initializing launch client");
|
||||
var launchServer = HostContext.GetService<ILaunchServer>();
|
||||
launchServer.InitializeLaunchClient(new Uri(launchReceiverEndpoint), accessToken);
|
||||
}
|
||||
_jobServerQueue = HostContext.GetService<IJobServerQueue>();
|
||||
_jobServerQueue.Start(message, resultServiceOnly: true);
|
||||
}
|
||||
else
|
||||
{
|
||||
@@ -120,8 +97,7 @@ namespace GitHub.Runner.Worker
|
||||
default:
|
||||
throw new ArgumentException(HostContext.RunnerShutdownReason.ToString(), nameof(HostContext.RunnerShutdownReason));
|
||||
}
|
||||
var issue = new Issue() { Type = IssueType.Error, Message = errorMessage };
|
||||
jobContext.AddIssue(issue, ExecutionContextLogOptions.Default);
|
||||
jobContext.AddIssue(new Issue() { Type = IssueType.Error, Message = errorMessage });
|
||||
});
|
||||
|
||||
// Validate directory permissions.
|
||||
@@ -285,7 +261,7 @@ namespace GitHub.Runner.Worker
|
||||
{
|
||||
try
|
||||
{
|
||||
await runServer.CompleteJobAsync(message.Plan.PlanId, message.JobId, result, jobContext.JobOutputs, jobContext.Global.StepsResult, jobContext.Global.JobAnnotations, default);
|
||||
await runServer.CompleteJobAsync(message.Plan.PlanId, message.JobId, result, jobContext.JobOutputs, jobContext.Global.StepsResult, default);
|
||||
return result;
|
||||
}
|
||||
catch (Exception ex)
|
||||
|
||||
@@ -184,33 +184,9 @@ namespace GitHub.Services.Common
|
||||
return settings;
|
||||
}
|
||||
|
||||
/// <summary>
|
||||
/// Gets or sets the maximum size allowed for response content buffering.
|
||||
/// </summary>
|
||||
[DefaultValue(c_defaultContentBufferSize)]
|
||||
public Int32 MaxContentBufferSize
|
||||
{
|
||||
get
|
||||
{
|
||||
return m_maxContentBufferSize;
|
||||
}
|
||||
set
|
||||
{
|
||||
ArgumentUtility.CheckForOutOfRange(value, nameof(value), 0, c_maxAllowedContentBufferSize);
|
||||
m_maxContentBufferSize = value;
|
||||
}
|
||||
}
|
||||
|
||||
private static Lazy<RawClientHttpRequestSettings> s_defaultSettings
|
||||
= new Lazy<RawClientHttpRequestSettings>(ConstructDefaultSettings);
|
||||
|
||||
private Int32 m_maxContentBufferSize;
|
||||
// We will buffer a maximum of 1024MB in the message handler
|
||||
private const Int32 c_maxAllowedContentBufferSize = 1024 * 1024 * 1024;
|
||||
|
||||
// We will buffer, by default, up to 512MB in the message handler
|
||||
private const Int32 c_defaultContentBufferSize = 1024 * 1024 * 512;
|
||||
|
||||
private const Int32 c_defaultMaxRetry = 3;
|
||||
private static readonly TimeSpan s_defaultTimeout = TimeSpan.FromSeconds(100); //default WebAPI timeout
|
||||
private ICollection<CultureInfo> m_acceptLanguages = new List<CultureInfo>();
|
||||
|
||||
@@ -9,7 +9,7 @@ using GitHub.Services.OAuth;
|
||||
|
||||
namespace GitHub.Services.Common
|
||||
{
|
||||
public class RawHttpMessageHandler : HttpMessageHandler
|
||||
public class RawHttpMessageHandler: HttpMessageHandler
|
||||
{
|
||||
public RawHttpMessageHandler(
|
||||
FederatedCredential credentials)
|
||||
@@ -120,7 +120,6 @@ namespace GitHub.Services.Common
|
||||
Boolean succeeded = false;
|
||||
HttpResponseMessageWrapper responseWrapper;
|
||||
|
||||
Boolean lastResponseDemandedProxyAuth = false;
|
||||
Int32 retries = m_maxAuthRetries;
|
||||
try
|
||||
{
|
||||
@@ -139,13 +138,7 @@ namespace GitHub.Services.Common
|
||||
|
||||
// Let's start with sending a token
|
||||
IssuedToken token = await m_tokenProvider.GetTokenAsync(null, tokenSource.Token).ConfigureAwait(false);
|
||||
ApplyToken(request, token, applyICredentialsToWebProxy: lastResponseDemandedProxyAuth);
|
||||
|
||||
// The WinHttpHandler will chunk any content that does not have a computed length which is
|
||||
// not what we want. By loading into a buffer up-front we bypass this behavior and there is
|
||||
// no difference in the normal HttpClientHandler behavior here since this is what they were
|
||||
// already doing.
|
||||
await BufferRequestContentAsync(request, tokenSource.Token).ConfigureAwait(false);
|
||||
ApplyToken(request, token);
|
||||
|
||||
// ConfigureAwait(false) enables the continuation to be run outside any captured
|
||||
// SyncronizationContext (such as ASP.NET's) which keeps things from deadlocking...
|
||||
@@ -154,8 +147,7 @@ namespace GitHub.Services.Common
|
||||
responseWrapper = new HttpResponseMessageWrapper(response);
|
||||
|
||||
var isUnAuthorized = responseWrapper.StatusCode == HttpStatusCode.Unauthorized;
|
||||
lastResponseDemandedProxyAuth = responseWrapper.StatusCode == HttpStatusCode.ProxyAuthenticationRequired;
|
||||
if (!isUnAuthorized && !lastResponseDemandedProxyAuth)
|
||||
if (!isUnAuthorized)
|
||||
{
|
||||
// Validate the token after it has been successfully authenticated with the server.
|
||||
m_tokenProvider?.ValidateToken(token, responseWrapper);
|
||||
@@ -219,42 +211,15 @@ namespace GitHub.Services.Common
|
||||
}
|
||||
}
|
||||
|
||||
private static async Task BufferRequestContentAsync(
|
||||
HttpRequestMessage request,
|
||||
CancellationToken cancellationToken)
|
||||
{
|
||||
if (request.Content != null &&
|
||||
request.Headers.TransferEncodingChunked != true)
|
||||
{
|
||||
Int64? contentLength = request.Content.Headers.ContentLength;
|
||||
if (contentLength == null)
|
||||
{
|
||||
await request.Content.LoadIntoBufferAsync().EnforceCancellation(cancellationToken).ConfigureAwait(false);
|
||||
}
|
||||
|
||||
// Explicitly turn off chunked encoding since we have computed the request content size
|
||||
request.Headers.TransferEncodingChunked = false;
|
||||
}
|
||||
}
|
||||
|
||||
private void ApplyToken(
|
||||
HttpRequestMessage request,
|
||||
IssuedToken token,
|
||||
bool applyICredentialsToWebProxy = false)
|
||||
IssuedToken token)
|
||||
{
|
||||
switch (token)
|
||||
{
|
||||
case null:
|
||||
return;
|
||||
case ICredentials credentialsToken:
|
||||
if (applyICredentialsToWebProxy)
|
||||
{
|
||||
HttpClientHandler httpClientHandler = m_transportHandler as HttpClientHandler;
|
||||
if (httpClientHandler != null && httpClientHandler.Proxy != null)
|
||||
{
|
||||
httpClientHandler.Proxy.Credentials = credentialsToken;
|
||||
}
|
||||
}
|
||||
m_credentialWrapper.InnerCredentials = credentialsToken;
|
||||
break;
|
||||
default:
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using System;
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Linq;
|
||||
using System.Net;
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using System;
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.ComponentModel;
|
||||
using System.Globalization;
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using System;
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.ComponentModel;
|
||||
using System.Linq;
|
||||
@@ -163,6 +163,7 @@ namespace GitHub.DistributedTask.ObjectTemplating
|
||||
message = $"{prefix} {message}";
|
||||
}
|
||||
|
||||
Errors.Add(message);
|
||||
TraceWriter.Error(message);
|
||||
}
|
||||
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using System;
|
||||
using System;
|
||||
using System.Runtime.Serialization;
|
||||
using GitHub.Services.Common;
|
||||
|
||||
@@ -1519,26 +1519,6 @@ namespace GitHub.DistributedTask.WebApi
|
||||
}
|
||||
}
|
||||
|
||||
[Serializable]
|
||||
[ExceptionMapping("0.0", "3.0", "TaskOrchestrationJobAlreadyAcquiredException", "GitHub.DistributedTask.WebApi.TaskOrchestrationJobAlreadyAcquiredException, GitHub.DistributedTask.WebApi, Version=14.0.0.0, Culture=neutral, PublicKeyToken=b03f5f7f11d50a3a")]
|
||||
public sealed class TaskOrchestrationJobAlreadyAcquiredException : DistributedTaskException
|
||||
{
|
||||
public TaskOrchestrationJobAlreadyAcquiredException(String message)
|
||||
: base(message)
|
||||
{
|
||||
}
|
||||
|
||||
public TaskOrchestrationJobAlreadyAcquiredException(String message, Exception innerException)
|
||||
: base(message, innerException)
|
||||
{
|
||||
}
|
||||
|
||||
private TaskOrchestrationJobAlreadyAcquiredException(SerializationInfo info, StreamingContext context)
|
||||
: base(info, context)
|
||||
{
|
||||
}
|
||||
}
|
||||
|
||||
[Serializable]
|
||||
[ExceptionMapping("0.0", "3.0", "TaskOrchestrationPlanSecurityException", "GitHub.DistributedTask.WebApi.TaskOrchestrationPlanSecurityException, GitHub.DistributedTask.WebApi, Version=14.0.0.0, Culture=neutral, PublicKeyToken=b03f5f7f11d50a3a")]
|
||||
public sealed class TaskOrchestrationPlanSecurityException : DistributedTaskException
|
||||
|
||||
@@ -1,48 +0,0 @@
|
||||
using GitHub.Services.WebApi;
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Runtime.Serialization;
|
||||
using Newtonsoft.Json;
|
||||
using System.Linq;
|
||||
|
||||
namespace GitHub.DistributedTask.WebApi
|
||||
{
|
||||
|
||||
public class ListRunnersResponse
|
||||
{
|
||||
public ListRunnersResponse()
|
||||
{
|
||||
}
|
||||
|
||||
public ListRunnersResponse(ListRunnersResponse responseToBeCloned)
|
||||
{
|
||||
this.TotalCount = responseToBeCloned.TotalCount;
|
||||
this.Runners = responseToBeCloned.Runners;
|
||||
}
|
||||
|
||||
[JsonProperty("total_count")]
|
||||
public int TotalCount
|
||||
{
|
||||
get;
|
||||
set;
|
||||
}
|
||||
|
||||
[JsonProperty("runners")]
|
||||
public List<Runner> Runners
|
||||
{
|
||||
get;
|
||||
set;
|
||||
}
|
||||
|
||||
public ListRunnersResponse Clone()
|
||||
{
|
||||
return new ListRunnersResponse(this);
|
||||
}
|
||||
|
||||
public List<TaskAgent> ToTaskAgents()
|
||||
{
|
||||
return Runners.Select(runner => new TaskAgent() { Name = runner.Name }).ToList();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -1,63 +0,0 @@
|
||||
using System;
|
||||
using Newtonsoft.Json;
|
||||
|
||||
namespace GitHub.DistributedTask.WebApi
|
||||
{
|
||||
public class Runner
|
||||
{
|
||||
|
||||
public class Authorization
|
||||
{
|
||||
/// <summary>
|
||||
/// The url to refresh tokens
|
||||
/// </summary>
|
||||
[JsonProperty("authorization_url")]
|
||||
public Uri AuthorizationUrl
|
||||
{
|
||||
get;
|
||||
internal set;
|
||||
}
|
||||
|
||||
/// <summary>
|
||||
/// The url to connect to to poll for messages
|
||||
/// </summary>
|
||||
[JsonProperty("server_url")]
|
||||
public string ServerUrl
|
||||
{
|
||||
get;
|
||||
internal set;
|
||||
}
|
||||
|
||||
/// <summary>
|
||||
/// The client id to use when connecting to the authorization_url
|
||||
/// </summary>
|
||||
[JsonProperty("client_id")]
|
||||
public string ClientId
|
||||
{
|
||||
get;
|
||||
internal set;
|
||||
}
|
||||
}
|
||||
|
||||
[JsonProperty("name")]
|
||||
public string Name
|
||||
{
|
||||
get;
|
||||
internal set;
|
||||
}
|
||||
|
||||
[JsonProperty("id")]
|
||||
public Int32 Id
|
||||
{
|
||||
get;
|
||||
internal set;
|
||||
}
|
||||
|
||||
[JsonProperty("authorization")]
|
||||
public Authorization RunnerAuthorization
|
||||
{
|
||||
get;
|
||||
internal set;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,98 +0,0 @@
|
||||
using GitHub.Services.WebApi;
|
||||
using System;
|
||||
using System.Runtime.Serialization;
|
||||
using System.ComponentModel;
|
||||
using System.Collections.Generic;
|
||||
using Newtonsoft.Json;
|
||||
using System.Linq;
|
||||
|
||||
namespace GitHub.DistributedTask.WebApi
|
||||
{
|
||||
/// <summary>
|
||||
/// An organization-level grouping of runners.
|
||||
/// </summary>
|
||||
[DataContract]
|
||||
public class RunnerGroup
|
||||
{
|
||||
internal RunnerGroup()
|
||||
{
|
||||
}
|
||||
|
||||
public RunnerGroup(String name)
|
||||
{
|
||||
this.Name = name;
|
||||
}
|
||||
|
||||
private RunnerGroup(RunnerGroup poolToBeCloned)
|
||||
{
|
||||
this.Id = poolToBeCloned.Id;
|
||||
this.IsHosted = poolToBeCloned.IsHosted;
|
||||
this.Name = poolToBeCloned.Name;
|
||||
this.IsDefault = poolToBeCloned.IsDefault;
|
||||
}
|
||||
|
||||
[DataMember(EmitDefaultValue = false)]
|
||||
[JsonProperty("id")]
|
||||
public Int32 Id
|
||||
{
|
||||
get;
|
||||
set;
|
||||
}
|
||||
|
||||
[DataMember(EmitDefaultValue = false)]
|
||||
[JsonProperty("name")]
|
||||
public String Name
|
||||
{
|
||||
get;
|
||||
set;
|
||||
}
|
||||
|
||||
/// <summary>
|
||||
/// Gets or sets a value indicating whether or not this pool is internal and can't be modified by users
|
||||
/// </summary>
|
||||
[DataMember]
|
||||
[JsonProperty("default")]
|
||||
public bool IsDefault
|
||||
{
|
||||
get;
|
||||
set;
|
||||
}
|
||||
|
||||
/// <summary>
|
||||
/// Gets or sets a value indicating whether or not this pool is managed by the service.
|
||||
/// </summary>
|
||||
[DataMember]
|
||||
[JsonProperty("is_hosted")]
|
||||
public Boolean IsHosted
|
||||
{
|
||||
get;
|
||||
set;
|
||||
}
|
||||
}
|
||||
|
||||
public class RunnerGroupList
|
||||
{
|
||||
public RunnerGroupList()
|
||||
{
|
||||
this.RunnerGroups = new List<RunnerGroup>();
|
||||
}
|
||||
|
||||
public List<TaskAgentPool> ToAgentPoolList()
|
||||
{
|
||||
var agentPools = this.RunnerGroups.Select(x => new TaskAgentPool(x.Name)
|
||||
{
|
||||
Id = x.Id,
|
||||
IsHosted = x.IsHosted,
|
||||
IsInternal = x.IsDefault
|
||||
}).ToList();
|
||||
|
||||
return agentPools;
|
||||
}
|
||||
|
||||
[JsonProperty("runner_groups")]
|
||||
public List<RunnerGroup> RunnerGroups { get; set; }
|
||||
|
||||
[JsonProperty("total_count")]
|
||||
public int Count { get; set; }
|
||||
}
|
||||
}
|
||||
@@ -1,7 +1,8 @@
|
||||
using System;
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Linq;
|
||||
using System.Runtime.Serialization;
|
||||
using GitHub.Services.Common;
|
||||
|
||||
namespace GitHub.DistributedTask.WebApi
|
||||
{
|
||||
@@ -9,78 +10,69 @@ namespace GitHub.DistributedTask.WebApi
|
||||
public sealed class TimelineRecord
|
||||
{
|
||||
public TimelineRecord()
|
||||
: this(null)
|
||||
{
|
||||
this.Attempt = 1;
|
||||
}
|
||||
|
||||
private TimelineRecord(TimelineRecord recordToBeCloned)
|
||||
{
|
||||
this.EnsureInitialized();
|
||||
this.Attempt = recordToBeCloned.Attempt;
|
||||
this.ChangeId = recordToBeCloned.ChangeId;
|
||||
this.CurrentOperation = recordToBeCloned.CurrentOperation;
|
||||
this.FinishTime = recordToBeCloned.FinishTime;
|
||||
this.Id = recordToBeCloned.Id;
|
||||
this.Identifier = recordToBeCloned.Identifier;
|
||||
this.LastModified = recordToBeCloned.LastModified;
|
||||
this.Location = recordToBeCloned.Location;
|
||||
this.Name = recordToBeCloned.Name;
|
||||
this.Order = recordToBeCloned.Order;
|
||||
this.ParentId = recordToBeCloned.ParentId;
|
||||
this.PercentComplete = recordToBeCloned.PercentComplete;
|
||||
this.RecordType = recordToBeCloned.RecordType;
|
||||
this.Result = recordToBeCloned.Result;
|
||||
this.ResultCode = recordToBeCloned.ResultCode;
|
||||
this.StartTime = recordToBeCloned.StartTime;
|
||||
this.State = recordToBeCloned.State;
|
||||
this.TimelineId = recordToBeCloned.TimelineId;
|
||||
this.WorkerName = recordToBeCloned.WorkerName;
|
||||
this.RefName = recordToBeCloned.RefName;
|
||||
this.ErrorCount = recordToBeCloned.ErrorCount;
|
||||
this.WarningCount = recordToBeCloned.WarningCount;
|
||||
this.NoticeCount = recordToBeCloned.NoticeCount;
|
||||
this.AgentPlatform = recordToBeCloned.AgentPlatform;
|
||||
|
||||
if (recordToBeCloned != null)
|
||||
if (recordToBeCloned.Log != null)
|
||||
{
|
||||
this.Attempt = recordToBeCloned.Attempt;
|
||||
this.ChangeId = recordToBeCloned.ChangeId;
|
||||
this.CurrentOperation = recordToBeCloned.CurrentOperation;
|
||||
this.FinishTime = recordToBeCloned.FinishTime;
|
||||
this.Id = recordToBeCloned.Id;
|
||||
this.Identifier = recordToBeCloned.Identifier;
|
||||
this.LastModified = recordToBeCloned.LastModified;
|
||||
this.Location = recordToBeCloned.Location;
|
||||
this.Name = recordToBeCloned.Name;
|
||||
this.Order = recordToBeCloned.Order;
|
||||
this.ParentId = recordToBeCloned.ParentId;
|
||||
this.PercentComplete = recordToBeCloned.PercentComplete;
|
||||
this.RecordType = recordToBeCloned.RecordType;
|
||||
this.Result = recordToBeCloned.Result;
|
||||
this.ResultCode = recordToBeCloned.ResultCode;
|
||||
this.StartTime = recordToBeCloned.StartTime;
|
||||
this.State = recordToBeCloned.State;
|
||||
this.TimelineId = recordToBeCloned.TimelineId;
|
||||
this.WorkerName = recordToBeCloned.WorkerName;
|
||||
this.RefName = recordToBeCloned.RefName;
|
||||
this.ErrorCount = recordToBeCloned.ErrorCount;
|
||||
this.WarningCount = recordToBeCloned.WarningCount;
|
||||
this.NoticeCount = recordToBeCloned.NoticeCount;
|
||||
this.AgentPlatform = recordToBeCloned.AgentPlatform;
|
||||
|
||||
if (recordToBeCloned.Log != null)
|
||||
this.Log = new TaskLogReference
|
||||
{
|
||||
this.Log = new TaskLogReference
|
||||
{
|
||||
Id = recordToBeCloned.Log.Id,
|
||||
Location = recordToBeCloned.Log.Location,
|
||||
};
|
||||
}
|
||||
Id = recordToBeCloned.Log.Id,
|
||||
Location = recordToBeCloned.Log.Location,
|
||||
};
|
||||
}
|
||||
|
||||
if (recordToBeCloned.Details != null)
|
||||
if (recordToBeCloned.Details != null)
|
||||
{
|
||||
this.Details = new TimelineReference
|
||||
{
|
||||
this.Details = new TimelineReference
|
||||
{
|
||||
ChangeId = recordToBeCloned.Details.ChangeId,
|
||||
Id = recordToBeCloned.Details.Id,
|
||||
Location = recordToBeCloned.Details.Location,
|
||||
};
|
||||
}
|
||||
ChangeId = recordToBeCloned.Details.ChangeId,
|
||||
Id = recordToBeCloned.Details.Id,
|
||||
Location = recordToBeCloned.Details.Location,
|
||||
};
|
||||
}
|
||||
|
||||
if (recordToBeCloned.m_issues?.Count > 0)
|
||||
{
|
||||
this.Issues.AddRange(recordToBeCloned.Issues.Select(i => i.Clone()));
|
||||
}
|
||||
if (recordToBeCloned.m_issues?.Count> 0)
|
||||
{
|
||||
this.Issues.AddRange(recordToBeCloned.Issues.Select(i => i.Clone()));
|
||||
}
|
||||
|
||||
if (recordToBeCloned.m_previousAttempts?.Count > 0)
|
||||
{
|
||||
this.m_previousAttempts.AddRange(recordToBeCloned.m_previousAttempts);
|
||||
}
|
||||
if (recordToBeCloned.m_previousAttempts?.Count > 0)
|
||||
{
|
||||
this.PreviousAttempts.AddRange(recordToBeCloned.PreviousAttempts);
|
||||
}
|
||||
|
||||
if (recordToBeCloned.m_variables?.Count > 0)
|
||||
{
|
||||
// Don't pave over the case-insensitive Dictionary we initialized above.
|
||||
foreach (var kvp in recordToBeCloned.m_variables)
|
||||
{
|
||||
m_variables[kvp.Key] = kvp.Value.Clone();
|
||||
}
|
||||
}
|
||||
if (recordToBeCloned.m_variables?.Count > 0)
|
||||
{
|
||||
this.m_variables = recordToBeCloned.Variables.ToDictionary(k => k.Key, v => v.Value.Clone());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -106,14 +98,14 @@ namespace GitHub.DistributedTask.WebApi
|
||||
}
|
||||
|
||||
[DataMember(Name = "Type", Order = 3)]
|
||||
public string RecordType
|
||||
public String RecordType
|
||||
{
|
||||
get;
|
||||
set;
|
||||
}
|
||||
|
||||
[DataMember(Order = 4)]
|
||||
public string Name
|
||||
public String Name
|
||||
{
|
||||
get;
|
||||
set;
|
||||
@@ -134,7 +126,7 @@ namespace GitHub.DistributedTask.WebApi
|
||||
}
|
||||
|
||||
[DataMember(Order = 7)]
|
||||
public string CurrentOperation
|
||||
public String CurrentOperation
|
||||
{
|
||||
get;
|
||||
set;
|
||||
@@ -162,7 +154,7 @@ namespace GitHub.DistributedTask.WebApi
|
||||
}
|
||||
|
||||
[DataMember(Order = 11)]
|
||||
public string ResultCode
|
||||
public String ResultCode
|
||||
{
|
||||
get;
|
||||
set;
|
||||
@@ -183,7 +175,7 @@ namespace GitHub.DistributedTask.WebApi
|
||||
}
|
||||
|
||||
[DataMember(Order = 14)]
|
||||
public string WorkerName
|
||||
public String WorkerName
|
||||
{
|
||||
get;
|
||||
set;
|
||||
@@ -197,7 +189,7 @@ namespace GitHub.DistributedTask.WebApi
|
||||
}
|
||||
|
||||
[DataMember(Order = 16, EmitDefaultValue = false)]
|
||||
public string RefName
|
||||
public String RefName
|
||||
{
|
||||
get;
|
||||
set;
|
||||
@@ -217,46 +209,35 @@ namespace GitHub.DistributedTask.WebApi
|
||||
set;
|
||||
}
|
||||
|
||||
public Int32 ErrorCount
|
||||
[DataMember(Order = 40)]
|
||||
public Int32? ErrorCount
|
||||
{
|
||||
get
|
||||
{
|
||||
return m_errorCount.GetValueOrDefault(0);
|
||||
}
|
||||
set
|
||||
{
|
||||
m_errorCount = value;
|
||||
}
|
||||
get;
|
||||
set;
|
||||
}
|
||||
|
||||
public Int32 WarningCount
|
||||
[DataMember(Order = 50)]
|
||||
public Int32? WarningCount
|
||||
{
|
||||
get
|
||||
{
|
||||
return m_warningCount.GetValueOrDefault(0);
|
||||
}
|
||||
set
|
||||
{
|
||||
m_warningCount = value;
|
||||
}
|
||||
get;
|
||||
set;
|
||||
}
|
||||
|
||||
public Int32 NoticeCount
|
||||
[DataMember(Order = 55)]
|
||||
public Int32? NoticeCount
|
||||
{
|
||||
get
|
||||
{
|
||||
return m_noticeCount.GetValueOrDefault(0);
|
||||
}
|
||||
set
|
||||
{
|
||||
m_noticeCount = value;
|
||||
}
|
||||
get;
|
||||
set;
|
||||
}
|
||||
|
||||
public List<Issue> Issues
|
||||
{
|
||||
get
|
||||
{
|
||||
if (m_issues == null)
|
||||
{
|
||||
m_issues = new List<Issue>();
|
||||
}
|
||||
return m_issues;
|
||||
}
|
||||
}
|
||||
@@ -276,7 +257,7 @@ namespace GitHub.DistributedTask.WebApi
|
||||
}
|
||||
|
||||
[DataMember(Order = 131)]
|
||||
public string Identifier
|
||||
public String Identifier
|
||||
{
|
||||
get;
|
||||
set;
|
||||
@@ -293,14 +274,22 @@ namespace GitHub.DistributedTask.WebApi
|
||||
{
|
||||
get
|
||||
{
|
||||
if (m_previousAttempts == null)
|
||||
{
|
||||
m_previousAttempts = new List<TimelineAttempt>();
|
||||
}
|
||||
return m_previousAttempts;
|
||||
}
|
||||
}
|
||||
|
||||
public IDictionary<string, VariableValue> Variables
|
||||
public IDictionary<String, VariableValue> Variables
|
||||
{
|
||||
get
|
||||
{
|
||||
if (m_variables == null)
|
||||
{
|
||||
m_variables = new Dictionary<String, VariableValue>(StringComparer.OrdinalIgnoreCase);
|
||||
}
|
||||
return m_variables;
|
||||
}
|
||||
}
|
||||
@@ -310,53 +299,13 @@ namespace GitHub.DistributedTask.WebApi
|
||||
return new TimelineRecord(this);
|
||||
}
|
||||
|
||||
[OnDeserialized]
|
||||
private void OnDeserialized(StreamingContext context)
|
||||
{
|
||||
this.EnsureInitialized();
|
||||
}
|
||||
|
||||
|
||||
/// <summary>
|
||||
/// DataContractSerializer bypasses all constructor logic and inline initialization!
|
||||
/// This method takes the place of a workhorse constructor for baseline initialization.
|
||||
/// The expectation is for this logic to be accessible to constructors and also to the OnDeserialized helper.
|
||||
/// </summary>
|
||||
private void EnsureInitialized()
|
||||
{
|
||||
// Note that ?? is a short-circuiting operator. (??= would be preferable, but it's not supported in the .NET Framework version currently used by actions/runner.)
|
||||
|
||||
// De-nullify the following historically-nullable ints.
|
||||
// (After several weeks in production, it may be possible to eliminate these nullable backing fields.)
|
||||
m_errorCount = m_errorCount ?? 0;
|
||||
m_warningCount = m_warningCount ?? 0;
|
||||
m_noticeCount = m_noticeCount ?? 0;
|
||||
|
||||
m_issues = m_issues ?? new List<Issue>();
|
||||
m_previousAttempts = m_previousAttempts ?? new List<TimelineAttempt>();
|
||||
this.Attempt = Math.Max(this.Attempt, 1);
|
||||
|
||||
// Ensure whatever content may have been deserialized for m_variables is backed by a case-insensitive Dictionary.
|
||||
var empty = Enumerable.Empty<KeyValuePair<string, VariableValue>>();
|
||||
m_variables = new Dictionary<string, VariableValue>(m_variables ?? empty, StringComparer.OrdinalIgnoreCase);
|
||||
}
|
||||
|
||||
[DataMember(Name = nameof(ErrorCount), Order = 40)]
|
||||
private Int32? m_errorCount;
|
||||
|
||||
[DataMember(Name = nameof(WarningCount), Order = 50)]
|
||||
private Int32? m_warningCount;
|
||||
|
||||
[DataMember(Name = nameof(NoticeCount), Order = 55)]
|
||||
private Int32? m_noticeCount;
|
||||
|
||||
[DataMember(Name = nameof(Issues), EmitDefaultValue = false, Order = 60)]
|
||||
[DataMember(Name = "Issues", EmitDefaultValue = false, Order = 60)]
|
||||
private List<Issue> m_issues;
|
||||
|
||||
[DataMember(Name = nameof(Variables), EmitDefaultValue = false, Order = 80)]
|
||||
private Dictionary<string, VariableValue> m_variables;
|
||||
[DataMember(Name = "Variables", EmitDefaultValue = false, Order = 80)]
|
||||
private Dictionary<String, VariableValue> m_variables;
|
||||
|
||||
[DataMember(Name = nameof(PreviousAttempts), EmitDefaultValue = false, Order = 120)]
|
||||
[DataMember(Name = "PreviousAttempts", EmitDefaultValue = false, Order = 120)]
|
||||
private List<TimelineAttempt> m_previousAttempts;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,11 +1,13 @@
|
||||
using System.Collections.Generic;
|
||||
using System.Runtime.Serialization;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
|
||||
namespace GitHub.Actions.RunService.WebApi
|
||||
{
|
||||
[DataContract]
|
||||
public class AcquireJobRequest
|
||||
{
|
||||
[DataMember(Name = "jobMessageId", EmitDefaultValue = false)]
|
||||
public string JobMessageId { get; set; }
|
||||
[DataMember(Name = "streamId", EmitDefaultValue = false)]
|
||||
public string StreamID { get; set; }
|
||||
}
|
||||
}
|
||||
@@ -1,35 +0,0 @@
|
||||
using System.Runtime.Serialization;
|
||||
|
||||
namespace Sdk.RSWebApi.Contracts
|
||||
{
|
||||
[DataContract]
|
||||
public struct Annotation
|
||||
{
|
||||
[DataMember(Name = "level", EmitDefaultValue = false)]
|
||||
public AnnotationLevel Level;
|
||||
|
||||
[DataMember(Name = "message", EmitDefaultValue = false)]
|
||||
public string Message;
|
||||
|
||||
[DataMember(Name = "rawDetails", EmitDefaultValue = false)]
|
||||
public string RawDetails;
|
||||
|
||||
[DataMember(Name = "path", EmitDefaultValue = false)]
|
||||
public string Path;
|
||||
|
||||
[DataMember(Name = "isInfrastructureIssue", EmitDefaultValue = false)]
|
||||
public bool IsInfrastructureIssue;
|
||||
|
||||
[DataMember(Name = "startLine", EmitDefaultValue = false)]
|
||||
public long StartLine;
|
||||
|
||||
[DataMember(Name = "endLine", EmitDefaultValue = false)]
|
||||
public long EndLine;
|
||||
|
||||
[DataMember(Name = "startColumn", EmitDefaultValue = false)]
|
||||
public long StartColumn;
|
||||
|
||||
[DataMember(Name = "endColumn", EmitDefaultValue = false)]
|
||||
public long EndColumn;
|
||||
}
|
||||
}
|
||||
@@ -1,20 +0,0 @@
|
||||
using System.Runtime.Serialization;
|
||||
|
||||
namespace Sdk.RSWebApi.Contracts
|
||||
{
|
||||
[DataContract]
|
||||
public enum AnnotationLevel
|
||||
{
|
||||
[EnumMember]
|
||||
UNKNOWN = 0,
|
||||
|
||||
[EnumMember]
|
||||
NOTICE = 1,
|
||||
|
||||
[EnumMember]
|
||||
WARNING = 2,
|
||||
|
||||
[EnumMember]
|
||||
FAILURE = 3
|
||||
}
|
||||
}
|
||||
@@ -2,7 +2,6 @@ using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Runtime.Serialization;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using Sdk.RSWebApi.Contracts;
|
||||
|
||||
namespace GitHub.Actions.RunService.WebApi
|
||||
{
|
||||
@@ -23,8 +22,5 @@ namespace GitHub.Actions.RunService.WebApi
|
||||
|
||||
[DataMember(Name = "stepResults", EmitDefaultValue = false)]
|
||||
public IList<StepResult> StepResults { get; set; }
|
||||
|
||||
[DataMember(Name = "annotations", EmitDefaultValue = false)]
|
||||
public IList<Annotation> Annotations { get; set; }
|
||||
}
|
||||
}
|
||||
@@ -1,91 +0,0 @@
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
|
||||
namespace Sdk.RSWebApi.Contracts
|
||||
{
|
||||
public static class IssueExtensions
|
||||
{
|
||||
public static Annotation? ToAnnotation(this Issue issue)
|
||||
{
|
||||
var issueMessage = issue.Message;
|
||||
if (string.IsNullOrWhiteSpace(issueMessage))
|
||||
{
|
||||
if (!issue.Data.TryGetValue(RunIssueKeys.Message, out issueMessage) || string.IsNullOrWhiteSpace(issueMessage))
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
var annotationLevel = GetAnnotationLevel(issue.Type);
|
||||
var path = GetFilePath(issue);
|
||||
var lineNumber = GetAnnotationNumber(issue, RunIssueKeys.Line) ?? 0;
|
||||
var endLineNumber = GetAnnotationNumber(issue, RunIssueKeys.EndLine) ?? lineNumber;
|
||||
var columnNumber = GetAnnotationNumber(issue, RunIssueKeys.Col) ?? 0;
|
||||
var endColumnNumber = GetAnnotationNumber(issue, RunIssueKeys.EndColumn) ?? columnNumber;
|
||||
var logLineNumber = GetAnnotationNumber(issue, RunIssueKeys.LogLineNumber) ?? 0;
|
||||
|
||||
if (path == null && lineNumber == 0 && logLineNumber != 0)
|
||||
{
|
||||
lineNumber = logLineNumber;
|
||||
endLineNumber = logLineNumber;
|
||||
}
|
||||
|
||||
return new Annotation
|
||||
{
|
||||
Level = annotationLevel,
|
||||
Message = issueMessage,
|
||||
Path = path,
|
||||
StartLine = lineNumber,
|
||||
EndLine = endLineNumber,
|
||||
StartColumn = columnNumber,
|
||||
EndColumn = endColumnNumber,
|
||||
};
|
||||
}
|
||||
|
||||
private static AnnotationLevel GetAnnotationLevel(IssueType issueType)
|
||||
{
|
||||
switch (issueType)
|
||||
{
|
||||
case IssueType.Error:
|
||||
return AnnotationLevel.FAILURE;
|
||||
case IssueType.Warning:
|
||||
return AnnotationLevel.WARNING;
|
||||
case IssueType.Notice:
|
||||
return AnnotationLevel.NOTICE;
|
||||
default:
|
||||
return AnnotationLevel.UNKNOWN;
|
||||
}
|
||||
}
|
||||
|
||||
private static int? GetAnnotationNumber(Issue issue, string key)
|
||||
{
|
||||
if (issue.Data.TryGetValue(key, out var numberString) &&
|
||||
int.TryParse(numberString, out var number))
|
||||
{
|
||||
return number;
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
private static string GetAnnotationField(Issue issue, string key)
|
||||
{
|
||||
if (issue.Data.TryGetValue(key, out var value))
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
private static string GetFilePath(Issue issue)
|
||||
{
|
||||
if (issue.Data.TryGetValue(RunIssueKeys.File, out var path) &&
|
||||
!string.IsNullOrWhiteSpace(path))
|
||||
{
|
||||
return path;
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,13 +0,0 @@
|
||||
namespace Sdk.RSWebApi.Contracts
|
||||
{
|
||||
public static class RunIssueKeys
|
||||
{
|
||||
public const string Message = "message";
|
||||
public const string File = "file";
|
||||
public const string Line = "line";
|
||||
public const string Col = "col";
|
||||
public const string EndLine = "endLine";
|
||||
public const string EndColumn = "endColumn";
|
||||
public const string LogLineNumber = "logFileLineNumber";
|
||||
}
|
||||
}
|
||||
@@ -1,9 +1,7 @@
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Runtime.Serialization;
|
||||
using System.Threading.Tasks;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using Sdk.RSWebApi.Contracts;
|
||||
|
||||
namespace GitHub.Actions.RunService.WebApi
|
||||
{
|
||||
@@ -36,8 +34,5 @@ namespace GitHub.Actions.RunService.WebApi
|
||||
|
||||
[DataMember(Name = "completed_log_lines", EmitDefaultValue = false)]
|
||||
public long? CompletedLogLines { get; set; }
|
||||
|
||||
[DataMember(Name = "annotations", EmitDefaultValue = false)]
|
||||
public List<Annotation> Annotations { get; set; }
|
||||
}
|
||||
}
|
||||
@@ -1,6 +1,5 @@
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Net;
|
||||
using System.Net.Http;
|
||||
using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
@@ -56,7 +55,7 @@ namespace GitHub.Actions.RunService.WebApi
|
||||
{
|
||||
}
|
||||
|
||||
public async Task<AgentJobRequestMessage> GetJobMessageAsync(
|
||||
public Task<AgentJobRequestMessage> GetJobMessageAsync(
|
||||
Uri requestUri,
|
||||
string messageId,
|
||||
CancellationToken cancellationToken = default)
|
||||
@@ -64,42 +63,26 @@ namespace GitHub.Actions.RunService.WebApi
|
||||
HttpMethod httpMethod = new HttpMethod("POST");
|
||||
var payload = new AcquireJobRequest
|
||||
{
|
||||
JobMessageId = messageId,
|
||||
StreamID = messageId
|
||||
};
|
||||
|
||||
requestUri = new Uri(requestUri, "acquirejob");
|
||||
|
||||
var requestContent = new ObjectContent<AcquireJobRequest>(payload, new VssJsonMediaTypeFormatter(true));
|
||||
var result = await SendAsync<AgentJobRequestMessage>(
|
||||
return SendAsync<AgentJobRequestMessage>(
|
||||
httpMethod,
|
||||
requestUri: requestUri,
|
||||
content: requestContent,
|
||||
cancellationToken: cancellationToken);
|
||||
|
||||
if (result.IsSuccess)
|
||||
{
|
||||
return result.Value;
|
||||
}
|
||||
|
||||
switch (result.StatusCode)
|
||||
{
|
||||
case HttpStatusCode.NotFound:
|
||||
throw new TaskOrchestrationJobNotFoundException($"Job message not found: {messageId}");
|
||||
case HttpStatusCode.Conflict:
|
||||
throw new TaskOrchestrationJobAlreadyAcquiredException($"Job message already acquired: {messageId}");
|
||||
default:
|
||||
throw new Exception($"Failed to get job message: {result.Error}");
|
||||
}
|
||||
}
|
||||
|
||||
public async Task CompleteJobAsync(
|
||||
public Task CompleteJobAsync(
|
||||
Uri requestUri,
|
||||
Guid planId,
|
||||
Guid jobId,
|
||||
TaskResult result,
|
||||
Dictionary<String, VariableValue> outputs,
|
||||
IList<StepResult> stepResults,
|
||||
IList<Annotation> jobAnnotations,
|
||||
CancellationToken cancellationToken = default)
|
||||
{
|
||||
HttpMethod httpMethod = new HttpMethod("POST");
|
||||
@@ -109,33 +92,20 @@ namespace GitHub.Actions.RunService.WebApi
|
||||
JobID = jobId,
|
||||
Conclusion = result,
|
||||
Outputs = outputs,
|
||||
StepResults = stepResults,
|
||||
Annotations = jobAnnotations
|
||||
StepResults = stepResults
|
||||
};
|
||||
|
||||
requestUri = new Uri(requestUri, "completejob");
|
||||
|
||||
var requestContent = new ObjectContent<CompleteJobRequest>(payload, new VssJsonMediaTypeFormatter(true));
|
||||
var response = await SendAsync(
|
||||
return SendAsync(
|
||||
httpMethod,
|
||||
requestUri,
|
||||
content: requestContent,
|
||||
cancellationToken: cancellationToken);
|
||||
if (response.IsSuccessStatusCode)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
switch (response.StatusCode)
|
||||
{
|
||||
case HttpStatusCode.NotFound:
|
||||
throw new TaskOrchestrationJobNotFoundException($"Job not found: {jobId}");
|
||||
default:
|
||||
throw new Exception($"Failed to complete job: {response.ReasonPhrase}");
|
||||
}
|
||||
}
|
||||
|
||||
public async Task<RenewJobResponse> RenewJobAsync(
|
||||
public Task<RenewJobResponse> RenewJobAsync(
|
||||
Uri requestUri,
|
||||
Guid planId,
|
||||
Guid jobId,
|
||||
@@ -151,24 +121,11 @@ namespace GitHub.Actions.RunService.WebApi
|
||||
requestUri = new Uri(requestUri, "renewjob");
|
||||
|
||||
var requestContent = new ObjectContent<RenewJobRequest>(payload, new VssJsonMediaTypeFormatter(true));
|
||||
var result = await SendAsync<RenewJobResponse>(
|
||||
return SendAsync<RenewJobResponse>(
|
||||
httpMethod,
|
||||
requestUri,
|
||||
content: requestContent,
|
||||
cancellationToken: cancellationToken);
|
||||
|
||||
if (result.IsSuccess)
|
||||
{
|
||||
return result.Value;
|
||||
}
|
||||
|
||||
switch (result.StatusCode)
|
||||
{
|
||||
case HttpStatusCode.NotFound:
|
||||
throw new TaskOrchestrationJobNotFoundException($"Job not found: {jobId}");
|
||||
default:
|
||||
throw new Exception($"Failed to renew job: {result.Error}");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -8,7 +8,7 @@
|
||||
<NoWarn>NU1701;NU1603</NoWarn>
|
||||
<Version>$(Version)</Version>
|
||||
<DefineConstants>TRACE</DefineConstants>
|
||||
<LangVersion>8.0</LangVersion>
|
||||
<LangVersion>7.3</LangVersion>
|
||||
<AllowUnsafeBlocks>true</AllowUnsafeBlocks>
|
||||
</PropertyGroup>
|
||||
|
||||
|
||||
@@ -1,92 +0,0 @@
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Net;
|
||||
using System.Net.Http;
|
||||
using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
using GitHub.DistributedTask.Pipelines;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using GitHub.Services.Common;
|
||||
using GitHub.Services.OAuth;
|
||||
using GitHub.Services.WebApi;
|
||||
using Sdk.RSWebApi.Contracts;
|
||||
using Sdk.WebApi.WebApi;
|
||||
|
||||
namespace GitHub.Actions.RunService.WebApi
|
||||
{
|
||||
public class BrokerHttpClient : RawHttpClientBase
|
||||
{
|
||||
public BrokerHttpClient(
|
||||
Uri baseUrl,
|
||||
VssOAuthCredential credentials)
|
||||
: base(baseUrl, credentials)
|
||||
{
|
||||
}
|
||||
|
||||
public BrokerHttpClient(
|
||||
Uri baseUrl,
|
||||
VssOAuthCredential credentials,
|
||||
RawClientHttpRequestSettings settings)
|
||||
: base(baseUrl, credentials, settings)
|
||||
{
|
||||
}
|
||||
|
||||
public BrokerHttpClient(
|
||||
Uri baseUrl,
|
||||
VssOAuthCredential credentials,
|
||||
params DelegatingHandler[] handlers)
|
||||
: base(baseUrl, credentials, handlers)
|
||||
{
|
||||
}
|
||||
|
||||
public BrokerHttpClient(
|
||||
Uri baseUrl,
|
||||
VssOAuthCredential credentials,
|
||||
RawClientHttpRequestSettings settings,
|
||||
params DelegatingHandler[] handlers)
|
||||
: base(baseUrl, credentials, settings, handlers)
|
||||
{
|
||||
}
|
||||
|
||||
public BrokerHttpClient(
|
||||
Uri baseUrl,
|
||||
HttpMessageHandler pipeline,
|
||||
Boolean disposeHandler)
|
||||
: base(baseUrl, pipeline, disposeHandler)
|
||||
{
|
||||
}
|
||||
|
||||
public async Task<TaskAgentMessage> GetRunnerMessageAsync(
|
||||
string runnerVersion,
|
||||
TaskAgentStatus? status,
|
||||
CancellationToken cancellationToken = default
|
||||
)
|
||||
{
|
||||
var requestUri = new Uri(Client.BaseAddress, "message");
|
||||
|
||||
List<KeyValuePair<string, string>> queryParams = new List<KeyValuePair<string, string>>();
|
||||
|
||||
if (status != null)
|
||||
{
|
||||
queryParams.Add("status", status.Value.ToString());
|
||||
}
|
||||
if (runnerVersion != null)
|
||||
{
|
||||
queryParams.Add("runnerVersion", runnerVersion);
|
||||
}
|
||||
|
||||
var result = await SendAsync<TaskAgentMessage>(
|
||||
new HttpMethod("GET"),
|
||||
requestUri: requestUri,
|
||||
queryParameters: queryParams,
|
||||
cancellationToken: cancellationToken);
|
||||
|
||||
if (result.IsSuccess)
|
||||
{
|
||||
return result.Value;
|
||||
}
|
||||
|
||||
throw new Exception($"Failed to get job message: {result.Error}");
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,4 +1,3 @@
|
||||
using System.Collections.Generic;
|
||||
using System.Runtime.Serialization;
|
||||
using Newtonsoft.Json;
|
||||
using Newtonsoft.Json.Serialization;
|
||||
@@ -127,57 +126,6 @@ namespace GitHub.Services.Results.Contracts
|
||||
public bool Ok;
|
||||
}
|
||||
|
||||
[DataContract]
|
||||
[JsonObject(NamingStrategyType = typeof(SnakeCaseNamingStrategy))]
|
||||
public class StepsUpdateRequest
|
||||
{
|
||||
[DataMember]
|
||||
public IEnumerable<Step> Steps;
|
||||
[DataMember]
|
||||
public long ChangeOrder;
|
||||
[DataMember]
|
||||
public string WorkflowJobRunBackendId;
|
||||
[DataMember]
|
||||
public string WorkflowRunBackendId;
|
||||
}
|
||||
|
||||
[DataContract]
|
||||
[JsonObject(NamingStrategyType = typeof(SnakeCaseNamingStrategy))]
|
||||
public class Step
|
||||
{
|
||||
[DataMember]
|
||||
public string ExternalId;
|
||||
[DataMember]
|
||||
public int Number;
|
||||
[DataMember]
|
||||
public string Name;
|
||||
[DataMember]
|
||||
public Status Status;
|
||||
[DataMember]
|
||||
public string StartedAt;
|
||||
[DataMember]
|
||||
public string CompletedAt;
|
||||
[DataMember]
|
||||
public Conclusion Conclusion;
|
||||
}
|
||||
|
||||
public enum Status
|
||||
{
|
||||
StatusUnknown = 0,
|
||||
StatusInProgress = 3,
|
||||
StatusPending = 5,
|
||||
StatusCompleted = 6
|
||||
}
|
||||
|
||||
public enum Conclusion
|
||||
{
|
||||
ConclusionUnknown = 0,
|
||||
ConclusionSuccess = 2,
|
||||
ConclusionFailure = 3,
|
||||
ConclusionCancelled = 4,
|
||||
ConclusionSkipped = 7,
|
||||
}
|
||||
|
||||
public static class BlobStorageTypes
|
||||
{
|
||||
public static readonly string AzureBlobStorage = "BLOB_STORAGE_TYPE_AZURE";
|
||||
|
||||
@@ -1,70 +0,0 @@
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Runtime.Serialization;
|
||||
|
||||
namespace GitHub.Services.Launch.Contracts
|
||||
{
|
||||
[DataContract]
|
||||
public class ActionReferenceRequest
|
||||
{
|
||||
[DataMember(EmitDefaultValue = false, Name = "action")]
|
||||
public string Action { get; set; }
|
||||
|
||||
[DataMember(EmitDefaultValue = false, Name = "version")]
|
||||
public string Version { get; set; }
|
||||
|
||||
[DataMember(EmitDefaultValue = false, Name = "path")]
|
||||
public string Path { get; set; }
|
||||
}
|
||||
|
||||
[DataContract]
|
||||
public class ActionReferenceRequestList
|
||||
{
|
||||
[DataMember(EmitDefaultValue = false, Name = "actions")]
|
||||
public IList<ActionReferenceRequest> Actions { get; set; }
|
||||
}
|
||||
|
||||
[DataContract]
|
||||
public class ActionDownloadInfoResponse
|
||||
{
|
||||
[DataMember(EmitDefaultValue = false, Name = "authentication")]
|
||||
public ActionDownloadAuthenticationResponse Authentication { get; set; }
|
||||
|
||||
[DataMember(EmitDefaultValue = false, Name = "name")]
|
||||
public string Name { get; set; }
|
||||
|
||||
[DataMember(EmitDefaultValue = false, Name = "resolved_name")]
|
||||
public string ResolvedName { get; set; }
|
||||
|
||||
[DataMember(EmitDefaultValue = false, Name = "resolved_sha")]
|
||||
public string ResolvedSha { get; set; }
|
||||
|
||||
[DataMember(EmitDefaultValue = false, Name = "tar_url")]
|
||||
public string TarUrl { get; set; }
|
||||
|
||||
[DataMember(EmitDefaultValue = false, Name = "version")]
|
||||
public string Version { get; set; }
|
||||
|
||||
[DataMember(EmitDefaultValue = false, Name = "zip_url")]
|
||||
public string ZipUrl { get; set; }
|
||||
}
|
||||
|
||||
[DataContract]
|
||||
public class ActionDownloadAuthenticationResponse
|
||||
{
|
||||
[DataMember(EmitDefaultValue = false, Name = "expires_at")]
|
||||
public DateTime ExpiresAt { get; set; }
|
||||
|
||||
[DataMember(EmitDefaultValue = false, Name = "token")]
|
||||
public string Token { get; set; }
|
||||
}
|
||||
|
||||
[DataContract]
|
||||
public class ActionDownloadInfoResponseCollection
|
||||
{
|
||||
/// <summary>A mapping of action specifications to their download information.</summary>
|
||||
/// <remarks>The key is the full name of the action plus version, e.g. "actions/checkout@v2".</remarks>
|
||||
[DataMember(EmitDefaultValue = false, Name = "actions")]
|
||||
public IDictionary<string, ActionDownloadInfoResponse> Actions { get; set; }
|
||||
}
|
||||
}
|
||||
@@ -1,115 +0,0 @@
|
||||
#nullable enable
|
||||
|
||||
using System;
|
||||
using System.Linq;
|
||||
using System.Net.Http;
|
||||
using System.Net.Http.Formatting;
|
||||
using System.Net.Http.Headers;
|
||||
using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using GitHub.Services.Launch.Contracts;
|
||||
|
||||
using Sdk.WebApi.WebApi;
|
||||
|
||||
namespace GitHub.Services.Launch.Client
|
||||
{
|
||||
public class LaunchHttpClient : RawHttpClientBase
|
||||
{
|
||||
public LaunchHttpClient(
|
||||
Uri baseUrl,
|
||||
HttpMessageHandler pipeline,
|
||||
string token,
|
||||
bool disposeHandler)
|
||||
: base(baseUrl, pipeline, disposeHandler)
|
||||
{
|
||||
m_token = token;
|
||||
m_launchServiceUrl = baseUrl;
|
||||
m_formatter = new JsonMediaTypeFormatter();
|
||||
}
|
||||
|
||||
public async Task<ActionDownloadInfoCollection> GetResolveActionsDownloadInfoAsync(Guid planId, Guid jobId, ActionReferenceList actionReferenceList, CancellationToken cancellationToken)
|
||||
{
|
||||
var GetResolveActionsDownloadInfoURLEndpoint = new Uri(m_launchServiceUrl, $"/actions/build/{planId.ToString()}/jobs/{jobId.ToString()}/runnerresolve/actions");
|
||||
return ToServerData(await GetLaunchSignedURLResponse<ActionReferenceRequestList, ActionDownloadInfoResponseCollection>(GetResolveActionsDownloadInfoURLEndpoint, ToGitHubData(actionReferenceList), cancellationToken));
|
||||
}
|
||||
|
||||
// Resolve Actions
|
||||
private async Task<T> GetLaunchSignedURLResponse<R, T>(Uri uri, R request, CancellationToken cancellationToken)
|
||||
{
|
||||
using (HttpRequestMessage requestMessage = new HttpRequestMessage(HttpMethod.Post, uri))
|
||||
{
|
||||
requestMessage.Headers.Authorization = new AuthenticationHeaderValue("Bearer", m_token);
|
||||
requestMessage.Headers.Accept.Add(MediaTypeWithQualityHeaderValue.Parse("application/json"));
|
||||
|
||||
using (HttpContent content = new ObjectContent<R>(request, m_formatter))
|
||||
{
|
||||
requestMessage.Content = content;
|
||||
using (var response = await SendAsync(requestMessage, HttpCompletionOption.ResponseContentRead, cancellationToken: cancellationToken))
|
||||
{
|
||||
return await ReadJsonContentAsync<T>(response, cancellationToken);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static ActionReferenceRequestList ToGitHubData(ActionReferenceList actionReferenceList)
|
||||
{
|
||||
return new ActionReferenceRequestList
|
||||
{
|
||||
Actions = actionReferenceList.Actions?.Select(ToGitHubData).ToList()
|
||||
};
|
||||
}
|
||||
|
||||
private static ActionReferenceRequest ToGitHubData(ActionReference actionReference)
|
||||
{
|
||||
return new ActionReferenceRequest
|
||||
{
|
||||
Action = actionReference.NameWithOwner,
|
||||
Version = actionReference.Ref,
|
||||
Path = actionReference.Path
|
||||
};
|
||||
}
|
||||
|
||||
private static ActionDownloadInfoCollection ToServerData(ActionDownloadInfoResponseCollection actionDownloadInfoResponseCollection)
|
||||
{
|
||||
return new ActionDownloadInfoCollection
|
||||
{
|
||||
Actions = actionDownloadInfoResponseCollection.Actions?.ToDictionary(kvp => kvp.Key, kvp => ToServerData(kvp.Value))
|
||||
};
|
||||
}
|
||||
|
||||
private static ActionDownloadInfo ToServerData(ActionDownloadInfoResponse actionDownloadInfoResponse)
|
||||
{
|
||||
return new ActionDownloadInfo
|
||||
{
|
||||
Authentication = ToServerData(actionDownloadInfoResponse.Authentication),
|
||||
NameWithOwner = actionDownloadInfoResponse.Name,
|
||||
ResolvedNameWithOwner = actionDownloadInfoResponse.ResolvedName,
|
||||
ResolvedSha = actionDownloadInfoResponse.ResolvedSha,
|
||||
TarballUrl = actionDownloadInfoResponse.TarUrl,
|
||||
Ref = actionDownloadInfoResponse.Version,
|
||||
ZipballUrl = actionDownloadInfoResponse.ZipUrl,
|
||||
};
|
||||
}
|
||||
|
||||
private static ActionDownloadAuthentication? ToServerData(ActionDownloadAuthenticationResponse? actionDownloadAuthenticationResponse)
|
||||
{
|
||||
if (actionDownloadAuthenticationResponse == null)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
return new ActionDownloadAuthentication
|
||||
{
|
||||
ExpiresAt = actionDownloadAuthenticationResponse.ExpiresAt,
|
||||
Token = actionDownloadAuthenticationResponse.Token
|
||||
};
|
||||
}
|
||||
|
||||
private MediaTypeFormatter m_formatter;
|
||||
private Uri m_launchServiceUrl;
|
||||
private string m_token;
|
||||
}
|
||||
}
|
||||
@@ -21,7 +21,7 @@ using Newtonsoft.Json.Linq;
|
||||
|
||||
namespace Sdk.WebApi.WebApi
|
||||
{
|
||||
public class RawHttpClientBase : IDisposable
|
||||
public class RawHttpClientBase: IDisposable
|
||||
{
|
||||
protected RawHttpClientBase(
|
||||
Uri baseUrl,
|
||||
@@ -101,7 +101,7 @@ namespace Sdk.WebApi.WebApi
|
||||
}
|
||||
}
|
||||
|
||||
protected Task<RawHttpClientResult<T>> SendAsync<T>(
|
||||
protected Task<T> SendAsync<T>(
|
||||
HttpMethod method,
|
||||
Uri requestUri,
|
||||
HttpContent content = null,
|
||||
@@ -112,7 +112,7 @@ namespace Sdk.WebApi.WebApi
|
||||
return SendAsync<T>(method, null, requestUri, content, queryParameters, userState, cancellationToken);
|
||||
}
|
||||
|
||||
protected async Task<RawHttpClientResult<T>> SendAsync<T>(
|
||||
protected async Task<T> SendAsync<T>(
|
||||
HttpMethod method,
|
||||
IEnumerable<KeyValuePair<String, String>> additionalHeaders,
|
||||
Uri requestUri,
|
||||
@@ -128,7 +128,7 @@ namespace Sdk.WebApi.WebApi
|
||||
}
|
||||
}
|
||||
|
||||
protected async Task<RawHttpClientResult<T>> SendAsync<T>(
|
||||
protected async Task<T> SendAsync<T>(
|
||||
HttpRequestMessage message,
|
||||
Object userState = null,
|
||||
CancellationToken cancellationToken = default(CancellationToken))
|
||||
@@ -138,16 +138,7 @@ namespace Sdk.WebApi.WebApi
|
||||
//from deadlocking...
|
||||
using (HttpResponseMessage response = await this.SendAsync(message, userState, cancellationToken).ConfigureAwait(false))
|
||||
{
|
||||
if (response.IsSuccessStatusCode)
|
||||
{
|
||||
T data = await ReadContentAsAsync<T>(response, cancellationToken).ConfigureAwait(false);
|
||||
return RawHttpClientResult<T>.Ok(data);
|
||||
}
|
||||
else
|
||||
{
|
||||
string errorMessage = $"Error: {response.ReasonPhrase}";
|
||||
return RawHttpClientResult<T>.Fail(errorMessage, response.StatusCode);
|
||||
}
|
||||
return await ReadContentAsAsync<T>(response, cancellationToken).ConfigureAwait(false);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -1,33 +0,0 @@
|
||||
using System.Net;
|
||||
|
||||
namespace Sdk.WebApi.WebApi
|
||||
{
|
||||
public class RawHttpClientResult
|
||||
{
|
||||
public bool IsSuccess { get; protected set; }
|
||||
public string Error { get; protected set; }
|
||||
public HttpStatusCode StatusCode { get; protected set; }
|
||||
public bool IsFailure => !IsSuccess;
|
||||
|
||||
protected RawHttpClientResult(bool isSuccess, string error, HttpStatusCode statusCode)
|
||||
{
|
||||
IsSuccess = isSuccess;
|
||||
Error = error;
|
||||
StatusCode = statusCode;
|
||||
}
|
||||
}
|
||||
|
||||
public class RawHttpClientResult<T> : RawHttpClientResult
|
||||
{
|
||||
public T Value { get; private set; }
|
||||
|
||||
protected internal RawHttpClientResult(T value, bool isSuccess, string error, HttpStatusCode statusCode)
|
||||
: base(isSuccess, error, statusCode)
|
||||
{
|
||||
Value = value;
|
||||
}
|
||||
|
||||
public static RawHttpClientResult<T> Fail(string message, HttpStatusCode statusCode) => new RawHttpClientResult<T>(default(T), false, message, statusCode);
|
||||
public static RawHttpClientResult<T> Ok(T value) => new RawHttpClientResult<T>(value, true, string.Empty, HttpStatusCode.OK);
|
||||
}
|
||||
}
|
||||
@@ -1,16 +1,11 @@
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Diagnostics;
|
||||
using System.IO;
|
||||
using System.Linq;
|
||||
using System.Net.Http;
|
||||
using System.Net.Http.Headers;
|
||||
using System.Threading;
|
||||
using System.Threading.Tasks;
|
||||
using System.Net.Http.Formatting;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using GitHub.Services.Common;
|
||||
using GitHub.Services.Results.Contracts;
|
||||
using System.Net.Http.Formatting;
|
||||
using Sdk.WebApi.WebApi;
|
||||
|
||||
namespace GitHub.Services.Results.Client
|
||||
@@ -27,7 +22,6 @@ namespace GitHub.Services.Results.Client
|
||||
m_token = token;
|
||||
m_resultsServiceUrl = baseUrl;
|
||||
m_formatter = new JsonMediaTypeFormatter();
|
||||
m_changeIdCounter = 1;
|
||||
}
|
||||
|
||||
// Get Sas URL calls
|
||||
@@ -92,7 +86,7 @@ namespace GitHub.Services.Results.Client
|
||||
|
||||
// Create metadata calls
|
||||
|
||||
private async Task SendRequest<R>(Uri uri, CancellationToken cancellationToken, R request, string timestamp)
|
||||
private async Task CreateMetadata<R>(Uri uri, CancellationToken cancellationToken, R request, string timestamp)
|
||||
{
|
||||
using (HttpRequestMessage requestMessage = new HttpRequestMessage(HttpMethod.Post, uri))
|
||||
{
|
||||
@@ -127,7 +121,7 @@ namespace GitHub.Services.Results.Client
|
||||
};
|
||||
|
||||
var createStepSummaryMetadataEndpoint = new Uri(m_resultsServiceUrl, Constants.CreateStepSummaryMetadata);
|
||||
await SendRequest<StepSummaryMetadataCreate>(createStepSummaryMetadataEndpoint, cancellationToken, request, timestamp);
|
||||
await CreateMetadata<StepSummaryMetadataCreate>(createStepSummaryMetadataEndpoint, cancellationToken, request, timestamp);
|
||||
}
|
||||
|
||||
private async Task StepLogUploadCompleteAsync(string planId, string jobId, Guid stepId, long lineCount, CancellationToken cancellationToken)
|
||||
@@ -143,7 +137,7 @@ namespace GitHub.Services.Results.Client
|
||||
};
|
||||
|
||||
var createStepLogsMetadataEndpoint = new Uri(m_resultsServiceUrl, Constants.CreateStepLogsMetadata);
|
||||
await SendRequest<StepLogsMetadataCreate>(createStepLogsMetadataEndpoint, cancellationToken, request, timestamp);
|
||||
await CreateMetadata<StepLogsMetadataCreate>(createStepLogsMetadataEndpoint, cancellationToken, request, timestamp);
|
||||
}
|
||||
|
||||
private async Task JobLogUploadCompleteAsync(string planId, string jobId, long lineCount, CancellationToken cancellationToken)
|
||||
@@ -158,7 +152,7 @@ namespace GitHub.Services.Results.Client
|
||||
};
|
||||
|
||||
var createJobLogsMetadataEndpoint = new Uri(m_resultsServiceUrl, Constants.CreateJobLogsMetadata);
|
||||
await SendRequest<JobLogsMetadataCreate>(createJobLogsMetadataEndpoint, cancellationToken, request, timestamp);
|
||||
await CreateMetadata<JobLogsMetadataCreate>(createJobLogsMetadataEndpoint, cancellationToken, request, timestamp);
|
||||
}
|
||||
|
||||
private async Task<HttpResponseMessage> UploadBlockFileAsync(string url, string blobStorageType, FileStream file, CancellationToken cancellationToken)
|
||||
@@ -320,81 +314,9 @@ namespace GitHub.Services.Results.Client
|
||||
}
|
||||
}
|
||||
|
||||
private Step ConvertTimelineRecordToStep(TimelineRecord r)
|
||||
{
|
||||
return new Step()
|
||||
{
|
||||
ExternalId = r.Id.ToString(),
|
||||
Number = r.Order.GetValueOrDefault(),
|
||||
Name = r.Name,
|
||||
Status = ConvertStateToStatus(r.State.GetValueOrDefault()),
|
||||
StartedAt = r.StartTime?.ToString(Constants.TimestampFormat),
|
||||
CompletedAt = r.FinishTime?.ToString(Constants.TimestampFormat),
|
||||
Conclusion = ConvertResultToConclusion(r.Result)
|
||||
};
|
||||
}
|
||||
|
||||
private Status ConvertStateToStatus(TimelineRecordState s)
|
||||
{
|
||||
switch (s)
|
||||
{
|
||||
case TimelineRecordState.Completed:
|
||||
return Status.StatusCompleted;
|
||||
case TimelineRecordState.Pending:
|
||||
return Status.StatusPending;
|
||||
case TimelineRecordState.InProgress:
|
||||
return Status.StatusInProgress;
|
||||
default:
|
||||
return Status.StatusUnknown;
|
||||
}
|
||||
}
|
||||
|
||||
private Conclusion ConvertResultToConclusion(TaskResult? r)
|
||||
{
|
||||
if (!r.HasValue)
|
||||
{
|
||||
return Conclusion.ConclusionUnknown;
|
||||
}
|
||||
|
||||
switch (r)
|
||||
{
|
||||
case TaskResult.Succeeded:
|
||||
case TaskResult.SucceededWithIssues:
|
||||
return Conclusion.ConclusionSuccess;
|
||||
case TaskResult.Canceled:
|
||||
return Conclusion.ConclusionCancelled;
|
||||
case TaskResult.Skipped:
|
||||
return Conclusion.ConclusionSkipped;
|
||||
case TaskResult.Failed:
|
||||
return Conclusion.ConclusionFailure;
|
||||
default:
|
||||
return Conclusion.ConclusionUnknown;
|
||||
}
|
||||
}
|
||||
|
||||
public async Task UpdateWorkflowStepsAsync(Guid planId, IEnumerable<TimelineRecord> records, CancellationToken cancellationToken)
|
||||
{
|
||||
var timestamp = DateTime.UtcNow.ToString(Constants.TimestampFormat);
|
||||
var stepRecords = records.Where(r => String.Equals(r.RecordType, "Task", StringComparison.Ordinal));
|
||||
var stepUpdateRequests = stepRecords.GroupBy(r => r.ParentId).Select(sg => new StepsUpdateRequest()
|
||||
{
|
||||
WorkflowRunBackendId = planId.ToString(),
|
||||
WorkflowJobRunBackendId = sg.Key.ToString(),
|
||||
ChangeOrder = m_changeIdCounter++,
|
||||
Steps = sg.Select(ConvertTimelineRecordToStep)
|
||||
});
|
||||
|
||||
var stepUpdateEndpoint = new Uri(m_resultsServiceUrl, Constants.WorkflowStepsUpdate);
|
||||
foreach (var request in stepUpdateRequests)
|
||||
{
|
||||
await SendRequest<StepsUpdateRequest>(stepUpdateEndpoint, cancellationToken, request, timestamp);
|
||||
}
|
||||
}
|
||||
|
||||
private MediaTypeFormatter m_formatter;
|
||||
private Uri m_resultsServiceUrl;
|
||||
private string m_token;
|
||||
private int m_changeIdCounter;
|
||||
}
|
||||
|
||||
// Constants specific to results
|
||||
@@ -409,8 +331,6 @@ namespace GitHub.Services.Results.Client
|
||||
public static readonly string CreateStepLogsMetadata = ResultsReceiverTwirpEndpoint + "CreateStepLogsMetadata";
|
||||
public static readonly string GetJobLogsSignedBlobURL = ResultsReceiverTwirpEndpoint + "GetJobLogsSignedBlobURL";
|
||||
public static readonly string CreateJobLogsMetadata = ResultsReceiverTwirpEndpoint + "CreateJobLogsMetadata";
|
||||
public static readonly string ResultsProtoApiV1Endpoint = "twirp/github.actions.results.api.v1.WorkflowStepUpdateService/";
|
||||
public static readonly string WorkflowStepsUpdate = ResultsProtoApiV1Endpoint + "WorkflowStepsUpdate";
|
||||
|
||||
public static readonly string AzureBlobSealedHeader = "x-ms-blob-sealed";
|
||||
public static readonly string AzureBlobTypeHeader = "x-ms-blob-type";
|
||||
|
||||
@@ -1,270 +0,0 @@
|
||||
#nullable enable
|
||||
|
||||
using System;
|
||||
using System.IO;
|
||||
using System.Collections.Generic;
|
||||
using System.Linq;
|
||||
using System.Runtime.Serialization.Json;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using Xunit;
|
||||
using System.Text;
|
||||
|
||||
namespace GitHub.Runner.Common.Tests.DistributedTask
|
||||
{
|
||||
public sealed class TimelineRecordL0
|
||||
{
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "DistributedTask")]
|
||||
public void VerifyTimelineRecord_Defaults()
|
||||
{
|
||||
var tr = new TimelineRecord();
|
||||
|
||||
Assert.Equal(0, tr.ErrorCount);
|
||||
Assert.Equal(0, tr.WarningCount);
|
||||
Assert.Equal(0, tr.NoticeCount);
|
||||
Assert.Equal(1, tr.Attempt);
|
||||
Assert.NotNull(tr.Issues);
|
||||
Assert.NotNull(tr.PreviousAttempts);
|
||||
Assert.NotNull(tr.Variables);
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "DistributedTask")]
|
||||
public void VerifyTimelineRecord_Clone()
|
||||
{
|
||||
var original = new TimelineRecord();
|
||||
original.ErrorCount = 100;
|
||||
original.WarningCount = 200;
|
||||
original.NoticeCount = 300;
|
||||
original.Attempt = 3;
|
||||
|
||||
// The Variables dictionary should be a case-insensitive dictionary.
|
||||
original.Variables["xxx"] = new VariableValue("first", false);
|
||||
original.Variables["XXX"] = new VariableValue("second", false);
|
||||
|
||||
Assert.Equal(1, original.Variables.Count);
|
||||
Assert.Equal("second", original.Variables.Values.First().Value);
|
||||
Assert.Equal("second", original.Variables["xXx"].Value);
|
||||
|
||||
var clone = original.Clone();
|
||||
|
||||
Assert.NotSame(original, clone);
|
||||
Assert.NotSame(original.Variables, clone.Variables);
|
||||
Assert.Equal(100, clone.ErrorCount);
|
||||
Assert.Equal(200, clone.WarningCount);
|
||||
Assert.Equal(300, clone.NoticeCount);
|
||||
Assert.Equal(3, clone.Attempt);
|
||||
|
||||
// Now, mutate the original post-clone.
|
||||
original.ErrorCount++;
|
||||
original.WarningCount += 10;
|
||||
original.NoticeCount *= 3;
|
||||
original.Attempt--;
|
||||
original.Variables["a"] = new VariableValue("1", false);
|
||||
|
||||
// Verify that the clone was unaffected by the changes to the original.
|
||||
Assert.Equal(100, clone.ErrorCount);
|
||||
Assert.Equal(200, clone.WarningCount);
|
||||
Assert.Equal(300, clone.NoticeCount);
|
||||
Assert.Equal(3, clone.Attempt);
|
||||
Assert.Equal(1, clone.Variables.Count);
|
||||
Assert.Equal("second", clone.Variables.Values.First().Value);
|
||||
|
||||
// Verify that the clone's Variables dictionary is also case-sensitive.
|
||||
clone.Variables["yyy"] = new VariableValue("third", false);
|
||||
clone.Variables["YYY"] = new VariableValue("fourth", false);
|
||||
|
||||
Assert.Equal(2, clone.Variables.Count);
|
||||
Assert.Equal("second", clone.Variables["xXx"].Value);
|
||||
Assert.Equal("fourth", clone.Variables["yYy"].Value);
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "DistributedTask")]
|
||||
public void VerifyTimelineRecord_DeserializationEdgeCase_NonNullCollections()
|
||||
{
|
||||
var jsonSamples = LoadJsonSamples(JsonSamplesFilePath);
|
||||
|
||||
// Verify that missing JSON fields don't result in null values for collection properties.
|
||||
var tr = Deserialize(jsonSamples["minimal"]);
|
||||
Assert.NotNull(tr);
|
||||
Assert.Equal("minimal", tr!.Name);
|
||||
Assert.NotNull(tr.Issues);
|
||||
Assert.NotNull(tr.PreviousAttempts);
|
||||
Assert.NotNull(tr.Variables);
|
||||
|
||||
// Verify that explicitly-null JSON fields don't result in null values for collection properties.
|
||||
// (Our deserialization logic should fix these up and instantiate an empty collection.)
|
||||
tr = Deserialize(jsonSamples["explicit-null-collections"]);
|
||||
Assert.NotNull(tr);
|
||||
Assert.Equal("explicit-null-collections", tr!.Name);
|
||||
Assert.NotNull(tr.Issues);
|
||||
Assert.NotNull(tr.PreviousAttempts);
|
||||
Assert.NotNull(tr.Variables);
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "DistributedTask")]
|
||||
public void VerifyTimelineRecord_DeserializationEdgeCase_AttemptCannotBeLessThan1()
|
||||
{
|
||||
var jsonSamples = LoadJsonSamples(JsonSamplesFilePath);
|
||||
|
||||
// Verify that 1 is the effective floor for TimelineRecord::Attempt.
|
||||
var tr = Deserialize(jsonSamples["minimal"]);
|
||||
Assert.NotNull(tr);
|
||||
Assert.Equal("minimal", tr!.Name);
|
||||
Assert.Equal(1, tr.Attempt);
|
||||
|
||||
tr = Deserialize(jsonSamples["invalid-attempt-value"]);
|
||||
Assert.NotNull(tr);
|
||||
Assert.Equal("invalid-attempt-value", tr!.Name);
|
||||
Assert.Equal(1, tr.Attempt);
|
||||
|
||||
tr = Deserialize(jsonSamples["zero-attempt-value"]);
|
||||
Assert.NotNull(tr);
|
||||
Assert.Equal("zero-attempt-value", tr!.Name);
|
||||
Assert.Equal(1, tr.Attempt);
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "DistributedTask")]
|
||||
public void VerifyTimelineRecord_DeserializationEdgeCase_HandleLegacyNullsGracefully()
|
||||
{
|
||||
var jsonSamples = LoadJsonSamples(JsonSamplesFilePath);
|
||||
|
||||
// Verify that nulls for ErrorCount, WarningCount, and NoticeCount are interpreted as 0.
|
||||
var tr = Deserialize(jsonSamples["legacy-nulls"]);
|
||||
Assert.NotNull(tr);
|
||||
Assert.Equal("legacy-nulls", tr!.Name);
|
||||
Assert.Equal(0, tr.ErrorCount);
|
||||
Assert.Equal(0, tr.WarningCount);
|
||||
Assert.Equal(0, tr.NoticeCount);
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "DistributedTask")]
|
||||
public void VerifyTimelineRecord_DeserializationEdgeCase_HandleMissingCountsGracefully()
|
||||
{
|
||||
var jsonSamples = LoadJsonSamples(JsonSamplesFilePath);
|
||||
|
||||
// Verify that nulls for ErrorCount, WarningCount, and NoticeCount are interpreted as 0.
|
||||
var tr = Deserialize(jsonSamples["missing-counts"]);
|
||||
Assert.NotNull(tr);
|
||||
Assert.Equal("missing-counts", tr!.Name);
|
||||
Assert.Equal(0, tr.ErrorCount);
|
||||
Assert.Equal(0, tr.WarningCount);
|
||||
Assert.Equal(0, tr.NoticeCount);
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "DistributedTask")]
|
||||
public void VerifyTimelineRecord_DeserializationEdgeCase_NonZeroCounts()
|
||||
{
|
||||
var jsonSamples = LoadJsonSamples(JsonSamplesFilePath);
|
||||
|
||||
// Verify that nulls for ErrorCount, WarningCount, and NoticeCount are interpreted as 0.
|
||||
var tr = Deserialize(jsonSamples["non-zero-counts"]);
|
||||
Assert.NotNull(tr);
|
||||
Assert.Equal("non-zero-counts", tr!.Name);
|
||||
Assert.Equal(10, tr.ErrorCount);
|
||||
Assert.Equal(20, tr.WarningCount);
|
||||
Assert.Equal(30, tr.NoticeCount);
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "DistributedTask")]
|
||||
public void VerifyTimelineRecord_Deserialization_LeanTimelineRecord()
|
||||
{
|
||||
var jsonSamples = LoadJsonSamples(JsonSamplesFilePath);
|
||||
|
||||
// Verify that a lean TimelineRecord can be deserialized.
|
||||
var tr = Deserialize(jsonSamples["lean"]);
|
||||
Assert.NotNull(tr);
|
||||
Assert.Equal("lean", tr!.Name);
|
||||
Assert.Equal(4, tr.Attempt);
|
||||
Assert.Equal(1, tr.Issues.Count);
|
||||
Assert.Equal(3, tr.Variables.Count);
|
||||
Assert.Equal(3, tr.PreviousAttempts.Count);
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "DistributedTask")]
|
||||
public void VerifyTimelineRecord_Deserialization_VariablesDictionaryIsCaseInsensitive()
|
||||
{
|
||||
var jsonSamples = LoadJsonSamples(JsonSamplesFilePath);
|
||||
|
||||
var tr = Deserialize(jsonSamples["lean"]);
|
||||
Assert.NotNull(tr);
|
||||
Assert.Equal("lean", tr!.Name);
|
||||
Assert.Equal(3, tr.Variables.Count);
|
||||
|
||||
// Verify that the Variables Dictionary is case-insensitive.
|
||||
tr.Variables["X"] = new VariableValue("overwritten", false);
|
||||
Assert.Equal(3, tr.Variables.Count);
|
||||
|
||||
tr.Variables["new"] = new VariableValue("new.1", false);
|
||||
Assert.Equal(4, tr.Variables.Count);
|
||||
|
||||
tr.Variables["NEW"] = new VariableValue("new.2", false);
|
||||
Assert.Equal(4, tr.Variables.Count);
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "DistributedTask")]
|
||||
public void VerifyTimelineRecord_DeserializationEdgeCase_DuplicateVariableKeysThrowsException()
|
||||
{
|
||||
var jsonSamples = LoadJsonSamples(JsonSamplesFilePath);
|
||||
|
||||
// We could be more forgiving in this case if we discover that it's not uncommon in Production for serialized TimelineRecords to:
|
||||
// 1) get incorrectly instantiated with a case-sensitive Variables dictionary (in older versions, this was possible via TimelineRecord::Clone)
|
||||
// 2) end up with case variations of the same key
|
||||
// 3) make another serialization/deserialization round trip.
|
||||
//
|
||||
// If we wanted to grant clemency to such incorrectly-serialized TimelineRecords,
|
||||
// the fix to TimelineRecord::EnsureInitialized would look something like the following:
|
||||
//
|
||||
// var seedVariables = m_variables ?? Enumerable.Empty<KeyValuePair<string, VariableValue>>();
|
||||
// m_variables = new Dictionary<string, VariableValue>(seedVariables.Count(), StringComparer.OrdinalIgnoreCase);
|
||||
// foreach (var kvp in seedVariables)
|
||||
// {
|
||||
// m_variables[kvp.Key] = kvp.Value;
|
||||
// }
|
||||
Assert.Throws<ArgumentException>(() => Deserialize(jsonSamples["duplicate-variable-keys"]));
|
||||
}
|
||||
|
||||
|
||||
private static Dictionary<string, string> LoadJsonSamples(string path)
|
||||
{
|
||||
// Embedding independent JSON samples within YML works well because JSON generally doesn't need to be escaped or otherwise mangled.
|
||||
var yamlDeserializer = new YamlDotNet.Serialization.Deserializer();
|
||||
using var stream = new StreamReader(path);
|
||||
return yamlDeserializer.Deserialize<Dictionary<string, string>>(stream);
|
||||
}
|
||||
|
||||
private static TimelineRecord? Deserialize(string rawJson)
|
||||
{
|
||||
using var stream = new MemoryStream(Encoding.UTF8.GetBytes(rawJson ?? string.Empty));
|
||||
return m_jsonSerializer.ReadObject(stream) as TimelineRecord;
|
||||
}
|
||||
|
||||
private static string JsonSamplesFilePath
|
||||
{
|
||||
get
|
||||
{
|
||||
return Path.Combine(TestUtil.GetTestDataPath(), "timelinerecord_json_samples.yml");
|
||||
}
|
||||
}
|
||||
|
||||
private static readonly DataContractJsonSerializer m_jsonSerializer = new(typeof(TimelineRecord));
|
||||
}
|
||||
}
|
||||
@@ -19,7 +19,6 @@ namespace GitHub.Runner.Common.Tests.Listener.Configuration
|
||||
public class ConfigurationManagerL0
|
||||
{
|
||||
private Mock<IRunnerServer> _runnerServer;
|
||||
private Mock<IRunnerDotcomServer> _dotcomServer;
|
||||
private Mock<ILocationServer> _locationServer;
|
||||
private Mock<ICredentialManager> _credMgr;
|
||||
private Mock<IPromptManager> _promptManager;
|
||||
@@ -56,7 +55,6 @@ namespace GitHub.Runner.Common.Tests.Listener.Configuration
|
||||
_store = new Mock<IConfigurationStore>();
|
||||
_extnMgr = new Mock<IExtensionManager>();
|
||||
_rsaKeyManager = new Mock<IRSAKeyManager>();
|
||||
_dotcomServer = new Mock<IRunnerDotcomServer>();
|
||||
|
||||
#if OS_WINDOWS
|
||||
_serviceControlManager = new Mock<IWindowsServiceControlManager>();
|
||||
@@ -73,13 +71,6 @@ namespace GitHub.Runner.Common.Tests.Listener.Configuration
|
||||
AuthorizationUrl = new Uri("http://localhost:8080/pipelines"),
|
||||
};
|
||||
|
||||
var expectedRunner = new GitHub.DistributedTask.WebApi.Runner() { Name = expectedAgent.Name, Id = 1 };
|
||||
expectedRunner.RunnerAuthorization = new GitHub.DistributedTask.WebApi.Runner.Authorization
|
||||
{
|
||||
ClientId = expectedAgent.Authorization.ClientId.ToString(),
|
||||
AuthorizationUrl = new Uri("http://localhost:8080/pipelines"),
|
||||
};
|
||||
|
||||
var connectionData = new ConnectionData()
|
||||
{
|
||||
InstanceId = Guid.NewGuid(),
|
||||
@@ -115,10 +106,6 @@ namespace GitHub.Runner.Common.Tests.Listener.Configuration
|
||||
_runnerServer.Setup(x => x.AddAgentAsync(It.IsAny<int>(), It.IsAny<TaskAgent>())).Returns(Task.FromResult(expectedAgent));
|
||||
_runnerServer.Setup(x => x.ReplaceAgentAsync(It.IsAny<int>(), It.IsAny<TaskAgent>())).Returns(Task.FromResult(expectedAgent));
|
||||
|
||||
_dotcomServer.Setup(x => x.GetRunnersAsync(It.IsAny<int>(), It.IsAny<string>(), It.IsAny<string>(), It.IsAny<string>())).Returns(Task.FromResult(expectedAgents));
|
||||
_dotcomServer.Setup(x => x.GetRunnerGroupsAsync(It.IsAny<string>(), It.IsAny<string>())).Returns(Task.FromResult(expectedPools));
|
||||
_dotcomServer.Setup(x => x.AddRunnerAsync(It.IsAny<int>(), It.IsAny<TaskAgent>(), It.IsAny<string>(), It.IsAny<string>(), It.IsAny<string>())).Returns(Task.FromResult(expectedRunner));
|
||||
|
||||
rsa = new RSACryptoServiceProvider(2048);
|
||||
|
||||
_rsaKeyManager.Setup(x => x.CreateKey()).Returns(rsa);
|
||||
@@ -132,7 +119,6 @@ namespace GitHub.Runner.Common.Tests.Listener.Configuration
|
||||
tc.SetSingleton<IConfigurationStore>(_store.Object);
|
||||
tc.SetSingleton<IExtensionManager>(_extnMgr.Object);
|
||||
tc.SetSingleton<IRunnerServer>(_runnerServer.Object);
|
||||
tc.SetSingleton<IRunnerDotcomServer>(_dotcomServer.Object);
|
||||
tc.SetSingleton<ILocationServer>(_locationServer.Object);
|
||||
|
||||
#if OS_WINDOWS
|
||||
@@ -206,107 +192,6 @@ namespace GitHub.Runner.Common.Tests.Listener.Configuration
|
||||
}
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "ConfigurationManagement")]
|
||||
public async Task ConfigureErrorDefaultLabelsDisabledWithNoCustomLabels()
|
||||
{
|
||||
using (TestHostContext tc = CreateTestContext())
|
||||
{
|
||||
Tracing trace = tc.GetTrace();
|
||||
|
||||
trace.Info("Creating config manager");
|
||||
IConfigurationManager configManager = new ConfigurationManager();
|
||||
configManager.Initialize(tc);
|
||||
|
||||
trace.Info("Preparing command line arguments");
|
||||
var command = new CommandSettings(
|
||||
tc,
|
||||
new[]
|
||||
{
|
||||
"configure",
|
||||
"--url", _expectedServerUrl,
|
||||
"--name", _expectedAgentName,
|
||||
"--runnergroup", _secondRunnerGroupName,
|
||||
"--work", _expectedWorkFolder,
|
||||
"--auth", _expectedAuthType,
|
||||
"--token", _expectedToken,
|
||||
"--no-default-labels",
|
||||
"--ephemeral",
|
||||
"--disableupdate",
|
||||
"--unattended",
|
||||
});
|
||||
trace.Info("Constructed.");
|
||||
_store.Setup(x => x.IsConfigured()).Returns(false);
|
||||
_configMgrAgentSettings = null;
|
||||
|
||||
trace.Info("Ensuring configure fails if default labels are disabled and no custom labels are set");
|
||||
var ex = await Assert.ThrowsAsync<NotSupportedException>(() => configManager.ConfigureAsync(command));
|
||||
|
||||
Assert.Contains("--no-default-labels without specifying --labels is not supported", ex.Message);
|
||||
}
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "ConfigurationManagement")]
|
||||
public async Task ConfigureDefaultLabelsDisabledWithCustomLabels()
|
||||
{
|
||||
using (TestHostContext tc = CreateTestContext())
|
||||
{
|
||||
Tracing trace = tc.GetTrace();
|
||||
|
||||
trace.Info("Creating config manager");
|
||||
IConfigurationManager configManager = new ConfigurationManager();
|
||||
configManager.Initialize(tc);
|
||||
|
||||
var userLabels = "userlabel1,userlabel2";
|
||||
|
||||
trace.Info("Preparing command line arguments");
|
||||
var command = new CommandSettings(
|
||||
tc,
|
||||
new[]
|
||||
{
|
||||
"configure",
|
||||
"--url", _expectedServerUrl,
|
||||
"--name", _expectedAgentName,
|
||||
"--runnergroup", _secondRunnerGroupName,
|
||||
"--work", _expectedWorkFolder,
|
||||
"--auth", _expectedAuthType,
|
||||
"--token", _expectedToken,
|
||||
"--labels", userLabels,
|
||||
"--no-default-labels",
|
||||
"--ephemeral",
|
||||
"--disableupdate",
|
||||
"--unattended",
|
||||
});
|
||||
trace.Info("Constructed.");
|
||||
_store.Setup(x => x.IsConfigured()).Returns(false);
|
||||
_configMgrAgentSettings = null;
|
||||
|
||||
trace.Info("Ensuring all the required parameters are available in the command line parameter");
|
||||
await configManager.ConfigureAsync(command);
|
||||
|
||||
_store.Setup(x => x.IsConfigured()).Returns(true);
|
||||
|
||||
trace.Info("Configured, verifying all the parameter value");
|
||||
var s = configManager.LoadSettings();
|
||||
Assert.NotNull(s);
|
||||
Assert.True(s.ServerUrl.Equals(_expectedServerUrl));
|
||||
Assert.True(s.AgentName.Equals(_expectedAgentName));
|
||||
Assert.True(s.PoolId.Equals(_secondRunnerGroupId));
|
||||
Assert.True(s.WorkFolder.Equals(_expectedWorkFolder));
|
||||
Assert.True(s.Ephemeral.Equals(true));
|
||||
|
||||
// validate GetAgentPoolsAsync gets called twice with automation pool type
|
||||
_runnerServer.Verify(x => x.GetAgentPoolsAsync(It.IsAny<string>(), It.Is<TaskAgentPoolType>(p => p == TaskAgentPoolType.Automation)), Times.Exactly(2));
|
||||
|
||||
var expectedLabels = userLabels.Split(",").ToList();
|
||||
|
||||
_runnerServer.Verify(x => x.AddAgentAsync(It.IsAny<int>(), It.Is<TaskAgent>(a => a.Labels.Select(x => x.Name).ToHashSet().SetEquals(expectedLabels))), Times.Once);
|
||||
}
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "ConfigurationManagement")]
|
||||
|
||||
@@ -1,61 +0,0 @@
|
||||
using System.Collections.Generic;
|
||||
using System.IO;
|
||||
using System.Runtime.Serialization.Json;
|
||||
using System.Text;
|
||||
using Xunit;
|
||||
|
||||
namespace GitHub.Actions.RunService.WebApi.Tests;
|
||||
|
||||
public sealed class AcquireJobRequestL0
|
||||
{
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "Common")]
|
||||
public void VerifySerialization()
|
||||
{
|
||||
var request = new AcquireJobRequest
|
||||
{
|
||||
JobMessageId = "1526919030369-33"
|
||||
};
|
||||
var serializer = new DataContractJsonSerializer(typeof(AcquireJobRequest));
|
||||
using var stream = new MemoryStream();
|
||||
serializer.WriteObject(stream, request);
|
||||
|
||||
stream.Position = 0;
|
||||
using var reader = new StreamReader(stream, Encoding.UTF8);
|
||||
string json = reader.ReadToEnd();
|
||||
string expected = DoubleQuotify(string.Format("{{'jobMessageId':'{0}'}}", request.JobMessageId));
|
||||
Assert.Equal(expected, json);
|
||||
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "Common")]
|
||||
public void VerifyDeserialization()
|
||||
{
|
||||
var serializer = new DataContractJsonSerializer(typeof(AcquireJobRequest));
|
||||
var variations = new Dictionary<string, string>()
|
||||
{
|
||||
["{'streamId': 'legacy', 'jobMessageId': 'new-1'}"] = "new-1",
|
||||
["{'jobMessageId': 'new-2', 'streamId': 'legacy'}"] = "new-2",
|
||||
["{'jobMessageId': 'new-3'}"] = "new-3"
|
||||
};
|
||||
|
||||
foreach (var (source, expected) in variations)
|
||||
{
|
||||
using var stream = new MemoryStream();
|
||||
stream.Write(Encoding.UTF8.GetBytes(DoubleQuotify(source)));
|
||||
stream.Position = 0;
|
||||
var recoveredRecord = serializer.ReadObject(stream) as AcquireJobRequest;
|
||||
Assert.NotNull(recoveredRecord);
|
||||
Assert.Equal(expected, recoveredRecord.JobMessageId);
|
||||
}
|
||||
}
|
||||
|
||||
private static string DoubleQuotify(string text)
|
||||
{
|
||||
return text.Replace('\'', '"');
|
||||
}
|
||||
}
|
||||
@@ -1,70 +0,0 @@
|
||||
using System.Collections.Generic;
|
||||
using GitHub.DistributedTask.WebApi;
|
||||
using Sdk.RSWebApi.Contracts;
|
||||
using Xunit;
|
||||
|
||||
namespace GitHub.Actions.RunService.WebApi.Tests;
|
||||
|
||||
public sealed class AnnotationsL0
|
||||
{
|
||||
[Fact]
|
||||
public void ToAnnotation_ValidIssueWithMessage_ReturnsAnnotation()
|
||||
{
|
||||
var issue = new Issue
|
||||
{
|
||||
Type = IssueType.Error,
|
||||
Message = "An error occurred",
|
||||
IsInfrastructureIssue = true
|
||||
};
|
||||
|
||||
issue.Data.Add(RunIssueKeys.File, "test.txt");
|
||||
issue.Data.Add(RunIssueKeys.Line, "5");
|
||||
issue.Data.Add(RunIssueKeys.Col, "10");
|
||||
issue.Data.Add(RunIssueKeys.EndLine, "8");
|
||||
issue.Data.Add(RunIssueKeys.EndColumn, "20");
|
||||
issue.Data.Add(RunIssueKeys.LogLineNumber, "2");
|
||||
|
||||
var annotation = issue.ToAnnotation();
|
||||
|
||||
Assert.NotNull(annotation);
|
||||
Assert.Equal(AnnotationLevel.FAILURE, annotation.Value.Level);
|
||||
Assert.Equal("An error occurred", annotation.Value.Message);
|
||||
Assert.Equal("test.txt", annotation.Value.Path);
|
||||
Assert.Equal(5, annotation.Value.StartLine);
|
||||
Assert.Equal(8, annotation.Value.EndLine);
|
||||
Assert.Equal(10, annotation.Value.StartColumn);
|
||||
Assert.Equal(20, annotation.Value.EndColumn);
|
||||
}
|
||||
|
||||
[Fact]
|
||||
public void ToAnnotation_ValidIssueWithEmptyMessage_ReturnsNull()
|
||||
{
|
||||
var issue = new Issue
|
||||
{
|
||||
Type = IssueType.Warning,
|
||||
Message = string.Empty
|
||||
};
|
||||
|
||||
var annotation = issue.ToAnnotation();
|
||||
|
||||
Assert.Null(annotation);
|
||||
}
|
||||
|
||||
[Fact]
|
||||
public void ToAnnotation_ValidIssueWithMessageInData_ReturnsAnnotation()
|
||||
{
|
||||
var issue = new Issue
|
||||
{
|
||||
Type = IssueType.Warning,
|
||||
Message = string.Empty,
|
||||
};
|
||||
|
||||
issue.Data.Add(RunIssueKeys.Message, "A warning occurred");
|
||||
|
||||
var annotation = issue.ToAnnotation();
|
||||
|
||||
Assert.NotNull(annotation);
|
||||
Assert.Equal(AnnotationLevel.WARNING, annotation.Value.Level);
|
||||
Assert.Equal("A warning occurred", annotation.Value.Message);
|
||||
}
|
||||
}
|
||||
@@ -1,3 +1,4 @@
|
||||
using GitHub.Runner.Common.Util;
|
||||
using GitHub.Runner.Sdk;
|
||||
using System;
|
||||
using System.IO;
|
||||
@@ -930,36 +931,6 @@ namespace GitHub.Runner.Common.Tests.Util
|
||||
}
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "Common")]
|
||||
public void LoadObject_ThrowsOnRequiredLoadObject()
|
||||
{
|
||||
using (TestHostContext hc = new(this))
|
||||
{
|
||||
Tracing trace = hc.GetTrace();
|
||||
|
||||
// Arrange: Create a directory with a file.
|
||||
string directory = Path.Combine(hc.GetDirectory(WellKnownDirectory.Bin), Path.GetRandomFileName());
|
||||
|
||||
string file = Path.Combine(directory, "empty file");
|
||||
Directory.CreateDirectory(directory);
|
||||
|
||||
File.WriteAllText(path: file, contents: "");
|
||||
Assert.Throws<ArgumentNullException>(() => IOUtil.LoadObject<RunnerSettings>(file, true));
|
||||
|
||||
file = Path.Combine(directory, "invalid type file");
|
||||
File.WriteAllText(path: file, contents: " ");
|
||||
Assert.Throws<ArgumentException>(() => IOUtil.LoadObject<RunnerSettings>(file, true));
|
||||
|
||||
// Cleanup.
|
||||
if (Directory.Exists(directory))
|
||||
{
|
||||
Directory.Delete(directory, recursive: true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static async Task CreateDirectoryReparsePoint(IHostContext context, string link, string target)
|
||||
{
|
||||
#if OS_WINDOWS
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using GitHub.Runner.Common.Util;
|
||||
using GitHub.Runner.Common.Util;
|
||||
using GitHub.Runner.Sdk;
|
||||
using System;
|
||||
using System.IO;
|
||||
@@ -89,128 +89,5 @@ namespace GitHub.Runner.Common.Tests.Util
|
||||
Assert.Equal(gitPath, gitPath2);
|
||||
}
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "Common")]
|
||||
public void WhichHandlesSymlinkToTargetFullPath()
|
||||
{
|
||||
// Arrange
|
||||
using TestHostContext hc = new TestHostContext(this);
|
||||
Tracing trace = hc.GetTrace();
|
||||
string oldValue = Environment.GetEnvironmentVariable(PathUtil.PathVariable);
|
||||
#if OS_WINDOWS
|
||||
string newValue = oldValue + @$";{Path.GetTempPath()}";
|
||||
string symlinkName = $"symlink-{Guid.NewGuid()}";
|
||||
string symlink = Path.GetTempPath() + $"{symlinkName}.exe";
|
||||
string target = Path.GetTempPath() + $"target-{Guid.NewGuid()}.exe";
|
||||
#else
|
||||
string newValue = oldValue + @$":{Path.GetTempPath()}";
|
||||
string symlinkName = $"symlink-{Guid.NewGuid()}";
|
||||
string symlink = Path.GetTempPath() + $"{symlinkName}";
|
||||
string target = Path.GetTempPath() + $"target-{Guid.NewGuid()}";
|
||||
#endif
|
||||
|
||||
Environment.SetEnvironmentVariable(PathUtil.PathVariable, newValue);
|
||||
|
||||
|
||||
using (File.Create(target))
|
||||
{
|
||||
File.CreateSymbolicLink(symlink, target);
|
||||
|
||||
// Act.
|
||||
var result = WhichUtil.Which(symlinkName, require: true, trace: trace);
|
||||
|
||||
// Assert
|
||||
Assert.True(!string.IsNullOrEmpty(result) && File.Exists(result), $"Unable to find symlink through: {nameof(WhichUtil.Which)}");
|
||||
|
||||
}
|
||||
|
||||
|
||||
// Cleanup
|
||||
File.Delete(symlink);
|
||||
File.Delete(target);
|
||||
Environment.SetEnvironmentVariable(PathUtil.PathVariable, oldValue);
|
||||
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "Common")]
|
||||
public void WhichHandlesSymlinkToTargetRelativePath()
|
||||
{
|
||||
// Arrange
|
||||
using TestHostContext hc = new TestHostContext(this);
|
||||
Tracing trace = hc.GetTrace();
|
||||
string oldValue = Environment.GetEnvironmentVariable(PathUtil.PathVariable);
|
||||
#if OS_WINDOWS
|
||||
string newValue = oldValue + @$";{Path.GetTempPath()}";
|
||||
string symlinkName = $"symlink-{Guid.NewGuid()}";
|
||||
string symlink = Path.GetTempPath() + $"{symlinkName}.exe";
|
||||
string targetName = $"target-{Guid.NewGuid()}.exe";
|
||||
string target = Path.GetTempPath() + targetName;
|
||||
#else
|
||||
string newValue = oldValue + @$":{Path.GetTempPath()}";
|
||||
string symlinkName = $"symlink-{Guid.NewGuid()}";
|
||||
string symlink = Path.GetTempPath() + $"{symlinkName}";
|
||||
string targetName = $"target-{Guid.NewGuid()}";
|
||||
string target = Path.GetTempPath() + targetName;
|
||||
#endif
|
||||
Environment.SetEnvironmentVariable(PathUtil.PathVariable, newValue);
|
||||
|
||||
|
||||
using (File.Create(target))
|
||||
{
|
||||
File.CreateSymbolicLink(symlink, targetName);
|
||||
|
||||
// Act.
|
||||
var result = WhichUtil.Which(symlinkName, require: true, trace: trace);
|
||||
|
||||
// Assert
|
||||
Assert.True(!string.IsNullOrEmpty(result) && File.Exists(result), $"Unable to find {symlinkName} through: {nameof(WhichUtil.Which)}");
|
||||
}
|
||||
|
||||
// Cleanup
|
||||
File.Delete(symlink);
|
||||
File.Delete(target);
|
||||
Environment.SetEnvironmentVariable(PathUtil.PathVariable, oldValue);
|
||||
|
||||
}
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "Common")]
|
||||
public void WhichThrowsWhenSymlinkBroken()
|
||||
{
|
||||
// Arrange
|
||||
using TestHostContext hc = new TestHostContext(this);
|
||||
Tracing trace = hc.GetTrace();
|
||||
string oldValue = Environment.GetEnvironmentVariable(PathUtil.PathVariable);
|
||||
|
||||
#if OS_WINDOWS
|
||||
string newValue = oldValue + @$";{Path.GetTempPath()}";
|
||||
string brokenSymlinkName = $"broken-symlink-{Guid.NewGuid()}";
|
||||
string brokenSymlink = Path.GetTempPath() + $"{brokenSymlinkName}.exe";
|
||||
#else
|
||||
string newValue = oldValue + @$":{Path.GetTempPath()}";
|
||||
string brokenSymlinkName = $"broken-symlink-{Guid.NewGuid()}";
|
||||
string brokenSymlink = Path.GetTempPath() + $"{brokenSymlinkName}";
|
||||
#endif
|
||||
|
||||
|
||||
string target = "no-such-file-cf7e351f";
|
||||
Environment.SetEnvironmentVariable(PathUtil.PathVariable, newValue);
|
||||
|
||||
File.CreateSymbolicLink(brokenSymlink, target);
|
||||
|
||||
// Act.
|
||||
var exception = Assert.Throws<FileNotFoundException>(()=>WhichUtil.Which(brokenSymlinkName, require: true, trace: trace));
|
||||
|
||||
// Assert
|
||||
Assert.Equal(brokenSymlinkName, exception.FileName);
|
||||
|
||||
// Cleanup
|
||||
File.Delete(brokenSymlink);
|
||||
Environment.SetEnvironmentVariable(PathUtil.PathVariable, oldValue);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using System;
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.IO;
|
||||
using System.Runtime.CompilerServices;
|
||||
@@ -32,10 +32,10 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
hc.GetTrace().Info($"{tag} {line}");
|
||||
return 1;
|
||||
});
|
||||
_ec.Setup(x => x.AddIssue(It.IsAny<Issue>(), It.IsAny<ExecutionContextLogOptions>()))
|
||||
.Callback((Issue issue, ExecutionContextLogOptions logOptions) =>
|
||||
_ec.Setup(x => x.AddIssue(It.IsAny<Issue>(), It.IsAny<string>()))
|
||||
.Callback((Issue issue, string message) =>
|
||||
{
|
||||
hc.GetTrace().Info($"{issue.Type} {issue.Message} {logOptions.LogMessageOverride ?? string.Empty}");
|
||||
hc.GetTrace().Info($"{issue.Type} {issue.Message} {message ?? string.Empty}");
|
||||
});
|
||||
|
||||
_commandManager.EnablePluginInternalCommand();
|
||||
@@ -59,10 +59,10 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
hc.GetTrace().Info($"{tag} {line}");
|
||||
return 1;
|
||||
});
|
||||
_ec.Setup(x => x.AddIssue(It.IsAny<Issue>(), It.IsAny<ExecutionContextLogOptions>()))
|
||||
.Callback((Issue issue, ExecutionContextLogOptions logOptions) =>
|
||||
_ec.Setup(x => x.AddIssue(It.IsAny<Issue>(), It.IsAny<string>()))
|
||||
.Callback((Issue issue, string message) =>
|
||||
{
|
||||
hc.GetTrace().Info($"{issue.Type} {issue.Message} {logOptions.LogMessageOverride ?? string.Empty}");
|
||||
hc.GetTrace().Info($"{issue.Type} {issue.Message} {message ?? string.Empty}");
|
||||
});
|
||||
|
||||
_commandManager.EnablePluginInternalCommand();
|
||||
@@ -92,10 +92,10 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
return 1;
|
||||
});
|
||||
|
||||
_ec.Setup(x => x.AddIssue(It.IsAny<Issue>(), It.IsAny<ExecutionContextLogOptions>()))
|
||||
.Callback((Issue issue, ExecutionContextLogOptions logOptions) =>
|
||||
_ec.Setup(x => x.AddIssue(It.IsAny<Issue>(), It.IsAny<string>()))
|
||||
.Callback((Issue issue, string message) =>
|
||||
{
|
||||
hc.GetTrace().Info($"{issue.Type} {issue.Message} {logOptions.LogMessageOverride ?? string.Empty}");
|
||||
hc.GetTrace().Info($"{issue.Type} {issue.Message} {message ?? string.Empty}");
|
||||
});
|
||||
|
||||
_ec.Object.Global.EnvironmentVariables = new Dictionary<string, string>();
|
||||
|
||||
@@ -29,7 +29,6 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
private Mock<IDockerCommandManager> _dockerManager;
|
||||
private Mock<IExecutionContext> _ec;
|
||||
private Mock<IJobServer> _jobServer;
|
||||
private Mock<ILaunchServer> _launchServer;
|
||||
private Mock<IRunnerPluginManager> _pluginManager;
|
||||
private TestHostContext _hc;
|
||||
private ActionManager _actionManager;
|
||||
@@ -2148,7 +2147,7 @@ runs:
|
||||
_ec.Object.Global.FileTable = new List<String>();
|
||||
_ec.Object.Global.Plan = new TaskOrchestrationPlanReference();
|
||||
_ec.Setup(x => x.Write(It.IsAny<string>(), It.IsAny<string>())).Callback((string tag, string message) => { _hc.GetTrace().Info($"[{tag}]{message}"); });
|
||||
_ec.Setup(x => x.AddIssue(It.IsAny<Issue>(), It.IsAny<ExecutionContextLogOptions>())).Callback((Issue issue, ExecutionContextLogOptions logOptions) => { _hc.GetTrace().Info($"[{issue.Type}]{logOptions.LogMessageOverride ?? issue.Message}"); });
|
||||
_ec.Setup(x => x.AddIssue(It.IsAny<Issue>(), It.IsAny<string>())).Callback((Issue issue, string message) => { _hc.GetTrace().Info($"[{issue.Type}]{issue.Message ?? message}"); });
|
||||
_ec.Setup(x => x.GetGitHubContext("workspace")).Returns(Path.Combine(_workFolder, "actions", "actions"));
|
||||
|
||||
_dockerManager = new Mock<IDockerCommandManager>();
|
||||
@@ -2176,25 +2175,6 @@ runs:
|
||||
return Task.FromResult(result);
|
||||
});
|
||||
|
||||
_launchServer = new Mock<ILaunchServer>();
|
||||
_launchServer.Setup(x => x.ResolveActionsDownloadInfoAsync(It.IsAny<Guid>(), It.IsAny<Guid>(), It.IsAny<ActionReferenceList>(), It.IsAny<CancellationToken>()))
|
||||
.Returns((Guid planId, Guid jobId, ActionReferenceList actions, CancellationToken cancellationToken) =>
|
||||
{
|
||||
var result = new ActionDownloadInfoCollection { Actions = new Dictionary<string, ActionDownloadInfo>() };
|
||||
foreach (var action in actions.Actions)
|
||||
{
|
||||
var key = $"{action.NameWithOwner}@{action.Ref}";
|
||||
result.Actions[key] = new ActionDownloadInfo
|
||||
{
|
||||
NameWithOwner = action.NameWithOwner,
|
||||
Ref = action.Ref,
|
||||
TarballUrl = $"https://api.github.com/repos/{action.NameWithOwner}/tarball/{action.Ref}",
|
||||
ZipballUrl = $"https://api.github.com/repos/{action.NameWithOwner}/zipball/{action.Ref}",
|
||||
};
|
||||
}
|
||||
return Task.FromResult(result);
|
||||
});
|
||||
|
||||
_pluginManager = new Mock<IRunnerPluginManager>();
|
||||
_pluginManager.Setup(x => x.GetPluginAction(It.IsAny<string>())).Returns(new RunnerPluginActionInfo() { PluginTypeName = "plugin.class, plugin", PostPluginTypeName = "plugin.cleanup, plugin" });
|
||||
|
||||
@@ -2203,7 +2183,6 @@ runs:
|
||||
|
||||
_hc.SetSingleton<IDockerCommandManager>(_dockerManager.Object);
|
||||
_hc.SetSingleton<IJobServer>(_jobServer.Object);
|
||||
_hc.SetSingleton<ILaunchServer>(_launchServer.Object);
|
||||
_hc.SetSingleton<IRunnerPluginManager>(_pluginManager.Object);
|
||||
_hc.SetSingleton<IActionManifestManager>(actionManifest);
|
||||
_hc.SetSingleton<IHttpClientHandlerFactory>(new HttpClientHandlerFactory());
|
||||
|
||||
@@ -715,7 +715,7 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
//Assert
|
||||
var err = Assert.Throws<ArgumentException>(() => actionManifest.Load(_ec.Object, action_path));
|
||||
Assert.Contains($"Fail to load {action_path}", err.Message);
|
||||
_ec.Verify(x => x.AddIssue(It.Is<Issue>(s => s.Message.Contains("Missing 'using' value. 'using' requires 'composite', 'docker', 'node12' or 'node16'.")), It.IsAny<ExecutionContextLogOptions>()), Times.Once);
|
||||
_ec.Verify(x => x.AddIssue(It.Is<Issue>(s => s.Message.Contains("Missing 'using' value. 'using' requires 'composite', 'docker', 'node12' or 'node16'.")), It.IsAny<string>()), Times.Once);
|
||||
}
|
||||
finally
|
||||
{
|
||||
@@ -860,7 +860,7 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
_ec.Setup(x => x.ExpressionValues).Returns(new DictionaryContextData());
|
||||
_ec.Setup(x => x.ExpressionFunctions).Returns(new List<IFunctionInfo>());
|
||||
_ec.Setup(x => x.Write(It.IsAny<string>(), It.IsAny<string>())).Callback((string tag, string message) => { _hc.GetTrace().Info($"{tag}{message}"); });
|
||||
_ec.Setup(x => x.AddIssue(It.IsAny<Issue>(), It.IsAny<ExecutionContextLogOptions>())).Callback((Issue issue, ExecutionContextLogOptions logOptions) => { _hc.GetTrace().Info($"[{issue.Type}]{logOptions.LogMessageOverride ?? issue.Message}"); });
|
||||
_ec.Setup(x => x.AddIssue(It.IsAny<Issue>(), It.IsAny<string>())).Callback((Issue issue, string message) => { _hc.GetTrace().Info($"[{issue.Type}]{issue.Message ?? message}"); });
|
||||
}
|
||||
|
||||
private void Teardown()
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using GitHub.DistributedTask.Expressions2;
|
||||
using GitHub.DistributedTask.Expressions2;
|
||||
using GitHub.DistributedTask.ObjectTemplating.Tokens;
|
||||
using GitHub.DistributedTask.Pipelines;
|
||||
using GitHub.DistributedTask.Pipelines.ContextData;
|
||||
@@ -366,7 +366,7 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
Assert.Equal("invalid1", finialInputs["invalid1"]);
|
||||
Assert.Equal("invalid2", finialInputs["invalid2"]);
|
||||
|
||||
_ec.Verify(x => x.AddIssue(It.Is<Issue>(s => s.Message.Contains("Unexpected input(s) 'invalid1', 'invalid2'")), It.IsAny<ExecutionContextLogOptions>()), Times.Once);
|
||||
_ec.Verify(x => x.AddIssue(It.Is<Issue>(s => s.Message.Contains("Unexpected input(s) 'invalid1', 'invalid2'")), It.IsAny<string>()), Times.Once);
|
||||
}
|
||||
|
||||
[Fact]
|
||||
@@ -485,7 +485,7 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
_ec.Setup(x => x.CancellationToken).Returns(_ecTokenSource.Token);
|
||||
_ec.Object.Global.Variables = new Variables(_hc, new Dictionary<string, VariableValue>());
|
||||
_ec.Setup(x => x.Write(It.IsAny<string>(), It.IsAny<string>())).Callback((string tag, string message) => { _hc.GetTrace().Info($"[{tag}]{message}"); });
|
||||
_ec.Setup(x => x.AddIssue(It.IsAny<Issue>(), It.IsAny<ExecutionContextLogOptions>())).Callback((Issue issue, ExecutionContextLogOptions logOptions) => { _hc.GetTrace().Info($"[{issue.Type}]{logOptions.LogMessageOverride ?? issue.Message}"); });
|
||||
_ec.Setup(x => x.AddIssue(It.IsAny<Issue>(), It.IsAny<string>())).Callback((Issue issue, string message) => { _hc.GetTrace().Info($"[{issue.Type}]{issue.Message ?? message}"); });
|
||||
|
||||
_hc.SetSingleton<IActionManager>(_actionManager.Object);
|
||||
_hc.SetSingleton<IHandlerFactory>(_handlerFactory.Object);
|
||||
|
||||
@@ -247,16 +247,12 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
WriteDebug = true,
|
||||
Variables = _variables,
|
||||
});
|
||||
_executionContext.Setup(x => x.AddIssue(It.IsAny<DTWebApi.Issue>(), It.IsAny<ExecutionContextLogOptions>()))
|
||||
.Callback((DTWebApi.Issue issue, ExecutionContextLogOptions logOptions) =>
|
||||
_executionContext.Setup(x => x.AddIssue(It.IsAny<DTWebApi.Issue>(), It.IsAny<string>()))
|
||||
.Callback((DTWebApi.Issue issue, string logMessage) =>
|
||||
{
|
||||
var resolvedMessage = issue.Message;
|
||||
if (logOptions.WriteToLog && !string.IsNullOrEmpty(logOptions.LogMessageOverride))
|
||||
{
|
||||
resolvedMessage = logOptions.LogMessageOverride;
|
||||
}
|
||||
_issues.Add(new(issue, resolvedMessage));
|
||||
_trace.Info($"Issue '{issue.Type}': {resolvedMessage}");
|
||||
_issues.Add(new Tuple<DTWebApi.Issue, string>(issue, logMessage));
|
||||
var message = !string.IsNullOrEmpty(logMessage) ? logMessage : issue.Message;
|
||||
_trace.Info($"Issue '{issue.Type}': {message}");
|
||||
});
|
||||
_executionContext.Setup(x => x.Write(It.IsAny<string>(), It.IsAny<string>()))
|
||||
.Callback((string tag, string message) =>
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using System;
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Linq;
|
||||
using System.Runtime.CompilerServices;
|
||||
@@ -52,43 +52,42 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
// Act.
|
||||
ec.InitializeJob(jobRequest, CancellationToken.None);
|
||||
|
||||
// Flood the ExecutionContext with errors and warnings (past its max capacity of 10).
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
|
||||
ec.Complete();
|
||||
|
||||
// Assert.
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.Is<TimelineRecord>(t => t.ErrorCount > 0)), Times.AtLeast(10));
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.Is<TimelineRecord>(t => t.WarningCount > 0)), Times.AtLeast(10));
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.Is<TimelineRecord>(t => t.ErrorCount == 15)), Times.AtLeastOnce);
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.Is<TimelineRecord>(t => t.WarningCount == 14)), Times.AtLeastOnce);
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.Is<TimelineRecord>(t => t.Issues.Where(i => i.Type == IssueType.Error).Count() == 10)), Times.AtLeastOnce);
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.Is<TimelineRecord>(t => t.Issues.Where(i => i.Type == IssueType.Warning).Count() == 10)), Times.AtLeastOnce);
|
||||
}
|
||||
@@ -191,9 +190,9 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
bigMessage += "a";
|
||||
}
|
||||
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = bigMessage }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = bigMessage }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Notice, Message = bigMessage }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = bigMessage });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = bigMessage });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Notice, Message = bigMessage });
|
||||
|
||||
ec.Complete();
|
||||
|
||||
@@ -204,61 +203,6 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
}
|
||||
}
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "Worker")]
|
||||
public void AddIssue_OverrideLogMessage()
|
||||
{
|
||||
using (TestHostContext hc = CreateTestContext())
|
||||
{
|
||||
// Arrange: Create a job request message.
|
||||
TaskOrchestrationPlanReference plan = new();
|
||||
TimelineReference timeline = new();
|
||||
Guid jobId = Guid.NewGuid();
|
||||
string jobName = "some job name";
|
||||
var jobRequest = new Pipelines.AgentJobRequestMessage(plan, timeline, jobId, jobName, jobName, null, null, null, new Dictionary<string, VariableValue>(), new List<MaskHint>(), new Pipelines.JobResources(), new Pipelines.ContextData.DictionaryContextData(), new Pipelines.WorkspaceOptions(), new List<Pipelines.ActionStep>(), null, null, null, null);
|
||||
jobRequest.Resources.Repositories.Add(new Pipelines.RepositoryResource()
|
||||
{
|
||||
Alias = Pipelines.PipelineConstants.SelfAlias,
|
||||
Id = "github",
|
||||
Version = "sha1"
|
||||
});
|
||||
jobRequest.ContextData["github"] = new Pipelines.ContextData.DictionaryContextData();
|
||||
|
||||
// Arrange: Setup the paging logger.
|
||||
var pagingLogger = new Mock<IPagingLogger>();
|
||||
var jobServerQueue = new Mock<IJobServerQueue>();
|
||||
|
||||
hc.EnqueueInstance(pagingLogger.Object);
|
||||
hc.SetSingleton(jobServerQueue.Object);
|
||||
|
||||
var ec = new Runner.Worker.ExecutionContext();
|
||||
ec.Initialize(hc);
|
||||
|
||||
// Act.
|
||||
ec.InitializeJob(jobRequest, CancellationToken.None);
|
||||
|
||||
var issueMessage = "Message embedded in issue.";
|
||||
var overrideMessage = "Message override.";
|
||||
var options = new ExecutionContextLogOptions(true, overrideMessage);
|
||||
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = issueMessage }, options);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = issueMessage }, options);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Notice, Message = issueMessage }, options);
|
||||
|
||||
// Finally, add a variation that DOESN'T override the message.
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Notice, Message = issueMessage }, ExecutionContextLogOptions.Default);
|
||||
|
||||
ec.Complete();
|
||||
|
||||
// Assert.
|
||||
jobServerQueue.Verify(x => x.QueueWebConsoleLine(It.IsAny<Guid>(), It.IsAny<string>(), It.IsAny<long?>()), Times.Exactly(4));
|
||||
jobServerQueue.Verify(x => x.QueueWebConsoleLine(It.IsAny<Guid>(), It.Is<string>(text => text.EndsWith(overrideMessage)), It.IsAny<long?>()), Times.Exactly(3));
|
||||
jobServerQueue.Verify(x => x.QueueWebConsoleLine(It.IsAny<Guid>(), It.Is<string>(text => text.EndsWith(issueMessage)), It.IsAny<long?>()), Times.Exactly(1));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
[Fact]
|
||||
[Trait("Level", "L0")]
|
||||
[Trait("Category", "Worker")]
|
||||
@@ -298,15 +242,13 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
var embeddedStep = ec.CreateChild(Guid.NewGuid(), "action_1_pre", "action_1_pre", null, null, ActionRunStage.Main, isEmbedded: true);
|
||||
embeddedStep.Start();
|
||||
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Error, Message = "error annotation that should have step and line number information" }, ExecutionContextLogOptions.Default);
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning annotation that should have step and line number information" }, ExecutionContextLogOptions.Default);
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Notice, Message = "notice annotation that should have step and line number information" }, ExecutionContextLogOptions.Default);
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Error, Message = "error annotation that should have step and line number information" });
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning annotation that should have step and line number information" });
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Notice, Message = "notice annotation that should have step and line number information" });
|
||||
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.IsAny<TimelineRecord>()), Times.AtLeastOnce);
|
||||
// Verify that Error/Warning/Notice issues added to embedded steps don't get sent up to the server.
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.Is<TimelineRecord>(t => t.Issues.Where(i => i.Data.ContainsKey("stepNumber") && i.Data.ContainsKey("logFileLineNumber") && i.Type == IssueType.Error).Count() == 1)), Times.Never);
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.Is<TimelineRecord>(t => t.Issues.Where(i => i.Data.ContainsKey("stepNumber") && i.Data.ContainsKey("logFileLineNumber") && i.Type == IssueType.Warning).Count() == 1)), Times.Never);
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.Is<TimelineRecord>(t => t.Issues.Where(i => i.Data.ContainsKey("stepNumber") && i.Data.ContainsKey("logFileLineNumber") && i.Type == IssueType.Notice).Count() == 1)), Times.Never);
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.Is<TimelineRecord>(t => t.Issues.Where(i => i.Data.ContainsKey("stepNumber") && i.Data.ContainsKey("logFileLineNumber") && i.Type == IssueType.Error).Count() == 1)), Times.AtLeastOnce);
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.Is<TimelineRecord>(t => t.Issues.Where(i => i.Data.ContainsKey("stepNumber") && i.Data.ContainsKey("logFileLineNumber") && i.Type == IssueType.Warning).Count() == 1)), Times.AtLeastOnce);
|
||||
jobServerQueue.Verify(x => x.QueueTimelineRecordUpdate(It.IsAny<Guid>(), It.Is<TimelineRecord>(t => t.Issues.Where(i => i.Data.ContainsKey("stepNumber") && i.Data.ContainsKey("logFileLineNumber") && i.Type == IssueType.Notice).Count() == 1)), Times.AtLeastOnce);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -684,12 +626,12 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
ec.StepTelemetry.StepId = Guid.NewGuid();
|
||||
ec.StepTelemetry.Stage = "main";
|
||||
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Notice, Message = "notice" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Notice, Message = "notice" }, ExecutionContextLogOptions.Default);
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Notice, Message = "notice" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
ec.AddIssue(new Issue() { Type = IssueType.Notice, Message = "notice" });
|
||||
|
||||
ec.Complete();
|
||||
|
||||
@@ -750,9 +692,9 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
embeddedStep.StepTelemetry.Action = "actions/checkout";
|
||||
embeddedStep.StepTelemetry.Ref = "v2";
|
||||
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Notice, Message = "notice" }, ExecutionContextLogOptions.Default);
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Notice, Message = "notice" });
|
||||
|
||||
embeddedStep.PublishStepTelemetry();
|
||||
|
||||
@@ -814,9 +756,9 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
embeddedStep.StepTelemetry.Action = "actions/checkout";
|
||||
embeddedStep.StepTelemetry.Ref = "v2";
|
||||
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" }, ExecutionContextLogOptions.Default);
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" }, ExecutionContextLogOptions.Default);
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Notice, Message = "notice" }, ExecutionContextLogOptions.Default);
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Error, Message = "error" });
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Warning, Message = "warning" });
|
||||
embeddedStep.AddIssue(new Issue() { Type = IssueType.Notice, Message = "notice" });
|
||||
|
||||
ec.Complete();
|
||||
|
||||
|
||||
@@ -984,15 +984,10 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
{
|
||||
_onMatcherChanged = handler;
|
||||
});
|
||||
_executionContext.Setup(x => x.AddIssue(It.IsAny<DTWebApi.Issue>(), It.IsAny<ExecutionContextLogOptions>()))
|
||||
.Callback((DTWebApi.Issue issue, ExecutionContextLogOptions logOptions) =>
|
||||
_executionContext.Setup(x => x.AddIssue(It.IsAny<DTWebApi.Issue>(), It.IsAny<string>()))
|
||||
.Callback((DTWebApi.Issue issue, string logMessage) =>
|
||||
{
|
||||
var resolvedMessage = issue.Message;
|
||||
if (logOptions.WriteToLog && !string.IsNullOrEmpty(logOptions.LogMessageOverride))
|
||||
{
|
||||
resolvedMessage = logOptions.LogMessageOverride;
|
||||
}
|
||||
_issues.Add(new(issue, resolvedMessage));
|
||||
_issues.Add(new Tuple<DTWebApi.Issue, string>(issue, logMessage));
|
||||
});
|
||||
_executionContext.Setup(x => x.Write(It.IsAny<string>(), It.IsAny<string>()))
|
||||
.Callback((string tag, string message) =>
|
||||
|
||||
@@ -413,16 +413,12 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
EnvironmentVariables = new Dictionary<string, string>(VarUtil.EnvironmentVariableKeyComparer),
|
||||
WriteDebug = true,
|
||||
});
|
||||
_executionContext.Setup(x => x.AddIssue(It.IsAny<DTWebApi.Issue>(), It.IsAny<ExecutionContextLogOptions>()))
|
||||
.Callback((DTWebApi.Issue issue, ExecutionContextLogOptions logOptions) =>
|
||||
_executionContext.Setup(x => x.AddIssue(It.IsAny<DTWebApi.Issue>(), It.IsAny<string>()))
|
||||
.Callback((DTWebApi.Issue issue, string logMessage) =>
|
||||
{
|
||||
var resolvedMessage = issue.Message;
|
||||
if (logOptions.WriteToLog && !string.IsNullOrEmpty(logOptions.LogMessageOverride))
|
||||
{
|
||||
resolvedMessage = logOptions.LogMessageOverride;
|
||||
}
|
||||
_issues.Add(new(issue, resolvedMessage));
|
||||
_trace.Info($"Issue '{issue.Type}': {resolvedMessage}");
|
||||
_issues.Add(new Tuple<DTWebApi.Issue, string>(issue, logMessage));
|
||||
var message = !string.IsNullOrEmpty(logMessage) ? logMessage : issue.Message;
|
||||
_trace.Info($"Issue '{issue.Type}': {message}");
|
||||
});
|
||||
_executionContext.Setup(x => x.Write(It.IsAny<string>(), It.IsAny<string>()))
|
||||
.Callback((string tag, string message) =>
|
||||
|
||||
@@ -411,16 +411,12 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
EnvironmentVariables = new Dictionary<string, string>(VarUtil.EnvironmentVariableKeyComparer),
|
||||
WriteDebug = true,
|
||||
});
|
||||
_executionContext.Setup(x => x.AddIssue(It.IsAny<DTWebApi.Issue>(), It.IsAny<ExecutionContextLogOptions>()))
|
||||
.Callback((DTWebApi.Issue issue, ExecutionContextLogOptions logOptions) =>
|
||||
_executionContext.Setup(x => x.AddIssue(It.IsAny<DTWebApi.Issue>(), It.IsAny<string>()))
|
||||
.Callback((DTWebApi.Issue issue, string logMessage) =>
|
||||
{
|
||||
var resolvedMessage = issue.Message;
|
||||
if (logOptions.WriteToLog && !string.IsNullOrEmpty(logOptions.LogMessageOverride))
|
||||
{
|
||||
resolvedMessage = logOptions.LogMessageOverride;
|
||||
}
|
||||
_issues.Add(new(issue, resolvedMessage));
|
||||
_trace.Info($"Issue '{issue.Type}': {resolvedMessage}");
|
||||
_issues.Add(new Tuple<DTWebApi.Issue, string>(issue, logMessage));
|
||||
var message = !string.IsNullOrEmpty(logMessage) ? logMessage : issue.Message;
|
||||
_trace.Info($"Issue '{issue.Type}': {message}");
|
||||
});
|
||||
_executionContext.Setup(x => x.Write(It.IsAny<string>(), It.IsAny<string>()))
|
||||
.Callback((string tag, string message) =>
|
||||
|
||||
@@ -413,16 +413,12 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
EnvironmentVariables = new Dictionary<string, string>(VarUtil.EnvironmentVariableKeyComparer),
|
||||
WriteDebug = true,
|
||||
});
|
||||
_executionContext.Setup(x => x.AddIssue(It.IsAny<DTWebApi.Issue>(), It.IsAny<ExecutionContextLogOptions>()))
|
||||
.Callback((DTWebApi.Issue issue, ExecutionContextLogOptions logOptions) =>
|
||||
_executionContext.Setup(x => x.AddIssue(It.IsAny<DTWebApi.Issue>(), It.IsAny<string>()))
|
||||
.Callback((DTWebApi.Issue issue, string logMessage) =>
|
||||
{
|
||||
var resolvedMessage = issue.Message;
|
||||
if (logOptions.WriteToLog && !string.IsNullOrEmpty(logOptions.LogMessageOverride))
|
||||
{
|
||||
resolvedMessage = logOptions.LogMessageOverride;
|
||||
}
|
||||
_issues.Add(new(issue, resolvedMessage));
|
||||
_trace.Info($"Issue '{issue.Type}': {resolvedMessage}");
|
||||
_issues.Add(new Tuple<DTWebApi.Issue, string>(issue, logMessage));
|
||||
var message = !string.IsNullOrEmpty(logMessage) ? logMessage : issue.Message;
|
||||
_trace.Info($"Issue '{issue.Type}': {message}");
|
||||
});
|
||||
_executionContext.Setup(x => x.Write(It.IsAny<string>(), It.IsAny<string>()))
|
||||
.Callback((string tag, string message) =>
|
||||
@@ -434,8 +430,8 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
_executionContext.Setup(x => x.SetOutput(It.IsAny<string>(), It.IsAny<string>(), out reference))
|
||||
.Callback((string name, string value, out string reference) =>
|
||||
{
|
||||
reference = value;
|
||||
_outputs[name] = value;
|
||||
reference = value;
|
||||
_outputs[name] = value;
|
||||
});
|
||||
|
||||
// SetOutputFileCommand
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
using System;
|
||||
using System;
|
||||
using System.Collections.Generic;
|
||||
using System.Globalization;
|
||||
using System.Linq;
|
||||
@@ -590,7 +590,7 @@ namespace GitHub.Runner.Common.Tests.Worker
|
||||
step.Setup(x => x.Condition).Returns(condition);
|
||||
step.Setup(x => x.ContinueOnError).Returns(new BooleanToken(null, null, null, continueOnError));
|
||||
step.Setup(x => x.Action)
|
||||
.Returns(new GitHub.DistributedTask.Pipelines.ActionStep()
|
||||
.Returns(new DistributedTask.Pipelines.ActionStep()
|
||||
{
|
||||
Name = name,
|
||||
Id = Guid.NewGuid(),
|
||||
|
||||
@@ -1,70 +0,0 @@
|
||||
minimal: |
|
||||
{ "Name": "minimal" }
|
||||
invalid-attempt-value: |
|
||||
{
|
||||
"Name": "invalid-attempt-value",
|
||||
"Attempt": -99
|
||||
}
|
||||
zero-attempt-value: |
|
||||
{
|
||||
"Name": "zero-attempt-value",
|
||||
"Attempt": 0
|
||||
}
|
||||
legacy-nulls: |
|
||||
{
|
||||
"Name": "legacy-nulls",
|
||||
"ErrorCount": null,
|
||||
"WarningCount": null,
|
||||
"NoticeCount": null
|
||||
}
|
||||
missing-counts: |
|
||||
{
|
||||
"Name": "missing-counts"
|
||||
}
|
||||
non-zero-counts: |
|
||||
{
|
||||
"Name": "non-zero-counts",
|
||||
"ErrorCount": 10,
|
||||
"WarningCount": 20,
|
||||
"NoticeCount": 30
|
||||
}
|
||||
explicit-null-collections: |
|
||||
{
|
||||
"Name": "explicit-null-collections",
|
||||
"Issues": null,
|
||||
"PreviousAttempts": null,
|
||||
"Variables": null
|
||||
}
|
||||
lean: |
|
||||
{
|
||||
"Id": "00000000-0000-0000-0000-000000000000",
|
||||
"Name": "lean",
|
||||
"LastModified": "\/Date(1679073003252+0000)\/",
|
||||
"Issues": [
|
||||
{
|
||||
"Type": 0,
|
||||
"Category": null,
|
||||
"Message": null,
|
||||
"IsInfrastructureIssue": null
|
||||
}
|
||||
],
|
||||
"Variables": [
|
||||
{ "Key": "x", "Value": { "Value": "1" } },
|
||||
{ "Key": "y", "Value": { "Value": "2" } },
|
||||
{ "Key": "z", "Value": { "Value": "3" } }
|
||||
],
|
||||
"Attempt": 4,
|
||||
"PreviousAttempts": [
|
||||
{ "Attempt": 1 },
|
||||
{ "Attempt": 2 },
|
||||
{ "Attempt": 3 }
|
||||
]
|
||||
}
|
||||
duplicate-variable-keys: |
|
||||
{
|
||||
"Name": "duplicate-variable-keys",
|
||||
"Variables": [
|
||||
{ "Key": "aaa", "Value": { "Value": "a.1" } },
|
||||
{ "Key": "AAA", "Value": { "Value": "a.2" } }
|
||||
]
|
||||
}
|
||||
@@ -1 +1 @@
|
||||
2.304.0
|
||||
2.303.0
|
||||
|
||||
Reference in New Issue
Block a user