Initial commit (history squashed)

This commit is contained in:
master
2025-10-07 10:14:21 +03:00
commit 016c5a3fe7
1132 changed files with 117842 additions and 0 deletions

View File

@@ -0,0 +1,39 @@
# AGENTS
## Role
Implement a connector for GitHub Security Advisories (GHSA) when we need to ingest GHSA content directly (instead of crosswalking via OSV/NVD).
## Scope
- Determine the optimal GHSA data source (GraphQL API, REST, or ecosystem export) and required authentication.
- Implement fetch logic with pagination, updated-since filtering, and cursor persistence.
- Parse GHSA records (identifiers, summaries, affected packages, versions, references, severity).
- Map advisories into canonical `Advisory` objects with aliases, references, affected packages, and range primitives.
- Provide deterministic fixtures and regression tests for the full pipeline.
## Participants
- `Source.Common` (HTTP clients, fetch service, DTO storage).
- `Storage.Mongo` (raw/document/DTO/advisory stores and source state).
- `Feedser.Models` (canonical advisory types).
- `Feedser.Testing` (integration harness, snapshot helpers).
## Interfaces & Contracts
- Job kinds: `ghsa:fetch`, `ghsa:parse`, `ghsa:map`.
- Support GitHub API authentication & rate limiting (token, retry/backoff).
- Alias set must include GHSA IDs and linked CVE IDs.
## In/Out of scope
In scope:
- Full GHSA connector implementation with range primitives and provenance instrumentation.
Out of scope:
- Repo-specific advisory ingest (handled via GitHub repo exports).
- Downstream ecosystem-specific enrichments.
## Observability & Security Expectations
- Log fetch pagination, throttling, and mapping stats.
- Handle GitHub API rate limits with exponential backoff and `Retry-After`.
- Sanitize/validate payloads before persistence.
## Tests
- Add `StellaOps.Feedser.Source.Ghsa.Tests` with canned GraphQL/REST fixtures.
- Snapshot canonical advisories; enable fixture regeneration with env flag.
- Confirm deterministic ordering/time normalisation.

View File

@@ -0,0 +1,61 @@
using System.Diagnostics.CodeAnalysis;
namespace StellaOps.Feedser.Source.Ghsa.Configuration;
public sealed class GhsaOptions
{
public static string HttpClientName => "source.ghsa";
public Uri BaseEndpoint { get; set; } = new("https://api.github.com/", UriKind.Absolute);
public string ApiToken { get; set; } = string.Empty;
public int PageSize { get; set; } = 50;
public int MaxPagesPerFetch { get; set; } = 5;
public TimeSpan InitialBackfill { get; set; } = TimeSpan.FromDays(30);
public TimeSpan RequestDelay { get; set; } = TimeSpan.FromMilliseconds(200);
public TimeSpan FailureBackoff { get; set; } = TimeSpan.FromMinutes(5);
[MemberNotNull(nameof(BaseEndpoint), nameof(ApiToken))]
public void Validate()
{
if (BaseEndpoint is null || !BaseEndpoint.IsAbsoluteUri)
{
throw new InvalidOperationException("BaseEndpoint must be an absolute URI.");
}
if (string.IsNullOrWhiteSpace(ApiToken))
{
throw new InvalidOperationException("ApiToken must be provided.");
}
if (PageSize is < 1 or > 100)
{
throw new InvalidOperationException("PageSize must be between 1 and 100.");
}
if (MaxPagesPerFetch <= 0)
{
throw new InvalidOperationException("MaxPagesPerFetch must be positive.");
}
if (InitialBackfill < TimeSpan.Zero)
{
throw new InvalidOperationException("InitialBackfill cannot be negative.");
}
if (RequestDelay < TimeSpan.Zero)
{
throw new InvalidOperationException("RequestDelay cannot be negative.");
}
if (FailureBackoff <= TimeSpan.Zero)
{
throw new InvalidOperationException("FailureBackoff must be greater than zero.");
}
}
}

View File

@@ -0,0 +1,394 @@
using System.Collections.Generic;
using System.Globalization;
using System.Linq;
using System.Net.Http;
using System.Text.Json;
using Microsoft.Extensions.Logging;
using Microsoft.Extensions.Options;
using MongoDB.Bson;
using StellaOps.Feedser.Models;
using StellaOps.Feedser.Source.Common;
using StellaOps.Feedser.Source.Common.Fetch;
using StellaOps.Feedser.Source.Ghsa.Configuration;
using StellaOps.Feedser.Source.Ghsa.Internal;
using StellaOps.Feedser.Storage.Mongo;
using StellaOps.Feedser.Storage.Mongo.Advisories;
using StellaOps.Feedser.Storage.Mongo.Documents;
using StellaOps.Feedser.Storage.Mongo.Dtos;
using StellaOps.Plugin;
namespace StellaOps.Feedser.Source.Ghsa;
public sealed class GhsaConnector : IFeedConnector
{
private static readonly JsonSerializerOptions SerializerOptions = new(JsonSerializerDefaults.Web)
{
PropertyNameCaseInsensitive = true,
WriteIndented = false,
};
private readonly SourceFetchService _fetchService;
private readonly RawDocumentStorage _rawDocumentStorage;
private readonly IDocumentStore _documentStore;
private readonly IDtoStore _dtoStore;
private readonly IAdvisoryStore _advisoryStore;
private readonly ISourceStateRepository _stateRepository;
private readonly GhsaOptions _options;
private readonly GhsaDiagnostics _diagnostics;
private readonly TimeProvider _timeProvider;
private readonly ILogger<GhsaConnector> _logger;
public GhsaConnector(
SourceFetchService fetchService,
RawDocumentStorage rawDocumentStorage,
IDocumentStore documentStore,
IDtoStore dtoStore,
IAdvisoryStore advisoryStore,
ISourceStateRepository stateRepository,
IOptions<GhsaOptions> options,
GhsaDiagnostics diagnostics,
TimeProvider? timeProvider,
ILogger<GhsaConnector> logger)
{
_fetchService = fetchService ?? throw new ArgumentNullException(nameof(fetchService));
_rawDocumentStorage = rawDocumentStorage ?? throw new ArgumentNullException(nameof(rawDocumentStorage));
_documentStore = documentStore ?? throw new ArgumentNullException(nameof(documentStore));
_dtoStore = dtoStore ?? throw new ArgumentNullException(nameof(dtoStore));
_advisoryStore = advisoryStore ?? throw new ArgumentNullException(nameof(advisoryStore));
_stateRepository = stateRepository ?? throw new ArgumentNullException(nameof(stateRepository));
_options = (options ?? throw new ArgumentNullException(nameof(options))).Value ?? throw new ArgumentNullException(nameof(options));
_options.Validate();
_diagnostics = diagnostics ?? throw new ArgumentNullException(nameof(diagnostics));
_timeProvider = timeProvider ?? TimeProvider.System;
_logger = logger ?? throw new ArgumentNullException(nameof(logger));
}
public string SourceName => GhsaConnectorPlugin.SourceName;
public async Task FetchAsync(IServiceProvider services, CancellationToken cancellationToken)
{
ArgumentNullException.ThrowIfNull(services);
var now = _timeProvider.GetUtcNow();
var cursor = await GetCursorAsync(cancellationToken).ConfigureAwait(false);
var pendingDocuments = cursor.PendingDocuments.ToHashSet();
var pendingMappings = cursor.PendingMappings.ToHashSet();
var since = cursor.CurrentWindowStart ?? cursor.LastUpdatedExclusive ?? now - _options.InitialBackfill;
if (since > now)
{
since = now;
}
var until = cursor.CurrentWindowEnd ?? now;
if (until <= since)
{
until = since + TimeSpan.FromMinutes(1);
}
var page = cursor.NextPage <= 0 ? 1 : cursor.NextPage;
var pagesFetched = 0;
var hasMore = true;
DateTimeOffset? maxUpdated = cursor.LastUpdatedExclusive;
while (hasMore && pagesFetched < _options.MaxPagesPerFetch)
{
cancellationToken.ThrowIfCancellationRequested();
var listUri = BuildListUri(since, until, page, _options.PageSize);
var metadata = new Dictionary<string, string>(StringComparer.Ordinal)
{
["since"] = since.ToString("O"),
["until"] = until.ToString("O"),
["page"] = page.ToString(CultureInfo.InvariantCulture),
["pageSize"] = _options.PageSize.ToString(CultureInfo.InvariantCulture),
};
SourceFetchContentResult listResult;
try
{
_diagnostics.FetchAttempt();
listResult = await _fetchService.FetchContentAsync(
new SourceFetchRequest(
GhsaOptions.HttpClientName,
SourceName,
listUri)
{
Metadata = metadata,
AcceptHeaders = new[] { "application/vnd.github+json" },
},
cancellationToken).ConfigureAwait(false);
}
catch (HttpRequestException ex)
{
_diagnostics.FetchFailure();
await _stateRepository.MarkFailureAsync(SourceName, now, _options.FailureBackoff, ex.Message, cancellationToken).ConfigureAwait(false);
throw;
}
if (listResult.IsNotModified)
{
_diagnostics.FetchUnchanged();
break;
}
if (!listResult.IsSuccess || listResult.Content is null)
{
_diagnostics.FetchFailure();
break;
}
var pageModel = GhsaListParser.Parse(listResult.Content, page, _options.PageSize);
if (pageModel.Items.Count == 0)
{
hasMore = false;
}
foreach (var item in pageModel.Items)
{
cancellationToken.ThrowIfCancellationRequested();
var detailUri = BuildDetailUri(item.GhsaId);
var detailMetadata = new Dictionary<string, string>(StringComparer.Ordinal)
{
["ghsaId"] = item.GhsaId,
["page"] = page.ToString(CultureInfo.InvariantCulture),
["since"] = since.ToString("O"),
["until"] = until.ToString("O"),
};
SourceFetchResult detailResult;
try
{
detailResult = await _fetchService.FetchAsync(
new SourceFetchRequest(
GhsaOptions.HttpClientName,
SourceName,
detailUri)
{
Metadata = detailMetadata,
AcceptHeaders = new[] { "application/vnd.github+json" },
},
cancellationToken).ConfigureAwait(false);
}
catch (HttpRequestException ex)
{
_diagnostics.FetchFailure();
_logger.LogWarning(ex, "Failed fetching GHSA advisory {GhsaId}", item.GhsaId);
continue;
}
if (detailResult.IsNotModified)
{
_diagnostics.FetchUnchanged();
continue;
}
if (!detailResult.IsSuccess || detailResult.Document is null)
{
_diagnostics.FetchFailure();
continue;
}
_diagnostics.FetchDocument();
pendingDocuments.Add(detailResult.Document.Id);
pendingMappings.Add(detailResult.Document.Id);
}
if (pageModel.MaxUpdated.HasValue)
{
if (!maxUpdated.HasValue || pageModel.MaxUpdated > maxUpdated)
{
maxUpdated = pageModel.MaxUpdated;
}
}
hasMore = pageModel.HasMorePages;
page = pageModel.NextPageCandidate;
pagesFetched++;
if (hasMore && _options.RequestDelay > TimeSpan.Zero)
{
await Task.Delay(_options.RequestDelay, cancellationToken).ConfigureAwait(false);
}
}
var updatedCursor = cursor
.WithPendingDocuments(pendingDocuments)
.WithPendingMappings(pendingMappings);
if (hasMore)
{
updatedCursor = updatedCursor
.WithCurrentWindowStart(since)
.WithCurrentWindowEnd(until)
.WithNextPage(page);
}
else
{
var nextSince = maxUpdated ?? until;
updatedCursor = updatedCursor
.WithLastUpdatedExclusive(nextSince)
.WithCurrentWindowStart(null)
.WithCurrentWindowEnd(null)
.WithNextPage(1);
}
await UpdateCursorAsync(updatedCursor, cancellationToken).ConfigureAwait(false);
}
public async Task ParseAsync(IServiceProvider services, CancellationToken cancellationToken)
{
ArgumentNullException.ThrowIfNull(services);
var cursor = await GetCursorAsync(cancellationToken).ConfigureAwait(false);
if (cursor.PendingDocuments.Count == 0)
{
return;
}
var remainingDocuments = cursor.PendingDocuments.ToList();
foreach (var documentId in cursor.PendingDocuments)
{
cancellationToken.ThrowIfCancellationRequested();
var document = await _documentStore.FindAsync(documentId, cancellationToken).ConfigureAwait(false);
if (document is null)
{
remainingDocuments.Remove(documentId);
continue;
}
if (!document.GridFsId.HasValue)
{
_diagnostics.ParseFailure();
_logger.LogWarning("GHSA document {DocumentId} missing GridFS content", documentId);
await _documentStore.UpdateStatusAsync(document.Id, DocumentStatuses.Failed, cancellationToken).ConfigureAwait(false);
remainingDocuments.Remove(documentId);
continue;
}
byte[] rawBytes;
try
{
rawBytes = await _rawDocumentStorage.DownloadAsync(document.GridFsId.Value, cancellationToken).ConfigureAwait(false);
}
catch (Exception ex)
{
_diagnostics.ParseFailure();
_logger.LogError(ex, "Unable to download GHSA raw document {DocumentId}", documentId);
throw;
}
GhsaRecordDto dto;
try
{
dto = GhsaRecordParser.Parse(rawBytes);
}
catch (JsonException ex)
{
_diagnostics.ParseQuarantine();
_logger.LogError(ex, "Malformed GHSA JSON for {DocumentId}", documentId);
await _documentStore.UpdateStatusAsync(document.Id, DocumentStatuses.Failed, cancellationToken).ConfigureAwait(false);
remainingDocuments.Remove(documentId);
continue;
}
var payload = BsonDocument.Parse(JsonSerializer.Serialize(dto, SerializerOptions));
var dtoRecord = new DtoRecord(
Guid.NewGuid(),
document.Id,
SourceName,
"ghsa/1.0",
payload,
_timeProvider.GetUtcNow());
await _dtoStore.UpsertAsync(dtoRecord, cancellationToken).ConfigureAwait(false);
await _documentStore.UpdateStatusAsync(document.Id, DocumentStatuses.PendingMap, cancellationToken).ConfigureAwait(false);
remainingDocuments.Remove(documentId);
_diagnostics.ParseSuccess();
}
var updatedCursor = cursor.WithPendingDocuments(remainingDocuments);
await UpdateCursorAsync(updatedCursor, cancellationToken).ConfigureAwait(false);
}
public async Task MapAsync(IServiceProvider services, CancellationToken cancellationToken)
{
ArgumentNullException.ThrowIfNull(services);
var cursor = await GetCursorAsync(cancellationToken).ConfigureAwait(false);
if (cursor.PendingMappings.Count == 0)
{
return;
}
var pendingMappings = cursor.PendingMappings.ToList();
foreach (var documentId in cursor.PendingMappings)
{
cancellationToken.ThrowIfCancellationRequested();
var dtoRecord = await _dtoStore.FindByDocumentIdAsync(documentId, cancellationToken).ConfigureAwait(false);
var document = await _documentStore.FindAsync(documentId, cancellationToken).ConfigureAwait(false);
if (dtoRecord is null || document is null)
{
_logger.LogWarning("Skipping GHSA mapping for {DocumentId}: DTO or document missing", documentId);
pendingMappings.Remove(documentId);
continue;
}
GhsaRecordDto dto;
try
{
dto = JsonSerializer.Deserialize<GhsaRecordDto>(dtoRecord.Payload.ToJson(), SerializerOptions)
?? throw new InvalidOperationException("Deserialized DTO was null.");
}
catch (Exception ex)
{
_logger.LogError(ex, "Failed to deserialize GHSA DTO for {DocumentId}", documentId);
await _documentStore.UpdateStatusAsync(document.Id, DocumentStatuses.Failed, cancellationToken).ConfigureAwait(false);
pendingMappings.Remove(documentId);
continue;
}
var advisory = GhsaMapper.Map(dto, document, dtoRecord.ValidatedAt);
await _advisoryStore.UpsertAsync(advisory, cancellationToken).ConfigureAwait(false);
await _documentStore.UpdateStatusAsync(document.Id, DocumentStatuses.Mapped, cancellationToken).ConfigureAwait(false);
pendingMappings.Remove(documentId);
_diagnostics.MapSuccess(1);
}
var updatedCursor = cursor.WithPendingMappings(pendingMappings);
await UpdateCursorAsync(updatedCursor, cancellationToken).ConfigureAwait(false);
}
private static Uri BuildListUri(DateTimeOffset since, DateTimeOffset until, int page, int pageSize)
{
var query = $"updated_since={Uri.EscapeDataString(since.ToString("O"))}&updated_until={Uri.EscapeDataString(until.ToString("O"))}&page={page}&per_page={pageSize}";
return new Uri($"security/advisories?{query}", UriKind.Relative);
}
private static Uri BuildDetailUri(string ghsaId)
{
var encoded = Uri.EscapeDataString(ghsaId);
return new Uri($"security/advisories/{encoded}", UriKind.Relative);
}
private async Task<GhsaCursor> GetCursorAsync(CancellationToken cancellationToken)
{
var state = await _stateRepository.TryGetAsync(SourceName, cancellationToken).ConfigureAwait(false);
return state is null ? GhsaCursor.Empty : GhsaCursor.FromBson(state.Cursor);
}
private async Task UpdateCursorAsync(GhsaCursor cursor, CancellationToken cancellationToken)
{
await _stateRepository.UpdateCursorAsync(SourceName, cursor.ToBsonDocument(), _timeProvider.GetUtcNow(), cancellationToken).ConfigureAwait(false);
}
}

View File

@@ -0,0 +1,19 @@
using Microsoft.Extensions.DependencyInjection;
using StellaOps.Plugin;
namespace StellaOps.Feedser.Source.Ghsa;
public sealed class GhsaConnectorPlugin : IConnectorPlugin
{
public const string SourceName = "ghsa";
public string Name => SourceName;
public bool IsAvailable(IServiceProvider services) => services is not null;
public IFeedConnector Create(IServiceProvider services)
{
ArgumentNullException.ThrowIfNull(services);
return ActivatorUtilities.CreateInstance<GhsaConnector>(services);
}
}

View File

@@ -0,0 +1,53 @@
using Microsoft.Extensions.Configuration;
using Microsoft.Extensions.DependencyInjection;
using StellaOps.DependencyInjection;
using StellaOps.Feedser.Core.Jobs;
using StellaOps.Feedser.Source.Ghsa.Configuration;
namespace StellaOps.Feedser.Source.Ghsa;
public sealed class GhsaDependencyInjectionRoutine : IDependencyInjectionRoutine
{
private const string ConfigurationSection = "feedser:sources:ghsa";
public IServiceCollection Register(IServiceCollection services, IConfiguration configuration)
{
ArgumentNullException.ThrowIfNull(services);
ArgumentNullException.ThrowIfNull(configuration);
services.AddGhsaConnector(options =>
{
configuration.GetSection(ConfigurationSection).Bind(options);
options.Validate();
});
services.AddTransient<GhsaFetchJob>();
services.AddTransient<GhsaParseJob>();
services.AddTransient<GhsaMapJob>();
services.PostConfigure<JobSchedulerOptions>(options =>
{
EnsureJob(options, GhsaJobKinds.Fetch, typeof(GhsaFetchJob));
EnsureJob(options, GhsaJobKinds.Parse, typeof(GhsaParseJob));
EnsureJob(options, GhsaJobKinds.Map, typeof(GhsaMapJob));
});
return services;
}
private static void EnsureJob(JobSchedulerOptions options, string kind, Type jobType)
{
if (options.Definitions.ContainsKey(kind))
{
return;
}
options.Definitions[kind] = new JobDefinition(
kind,
jobType,
options.DefaultTimeout,
options.DefaultLeaseDuration,
CronExpression: null,
Enabled: true);
}
}

View File

@@ -0,0 +1,37 @@
using Microsoft.Extensions.DependencyInjection;
using Microsoft.Extensions.Options;
using StellaOps.Feedser.Source.Common.Http;
using StellaOps.Feedser.Source.Ghsa.Configuration;
using StellaOps.Feedser.Source.Ghsa.Internal;
namespace StellaOps.Feedser.Source.Ghsa;
public static class GhsaServiceCollectionExtensions
{
public static IServiceCollection AddGhsaConnector(this IServiceCollection services, Action<GhsaOptions> configure)
{
ArgumentNullException.ThrowIfNull(services);
ArgumentNullException.ThrowIfNull(configure);
services.AddOptions<GhsaOptions>()
.Configure(configure)
.PostConfigure(static opts => opts.Validate());
services.AddSourceHttpClient(GhsaOptions.HttpClientName, (sp, clientOptions) =>
{
var options = sp.GetRequiredService<IOptions<GhsaOptions>>().Value;
clientOptions.BaseAddress = options.BaseEndpoint;
clientOptions.Timeout = TimeSpan.FromSeconds(30);
clientOptions.UserAgent = "StellaOps.Feedser.Ghsa/1.0";
clientOptions.AllowedHosts.Clear();
clientOptions.AllowedHosts.Add(options.BaseEndpoint.Host);
clientOptions.DefaultRequestHeaders["Accept"] = "application/vnd.github+json";
clientOptions.DefaultRequestHeaders["Authorization"] = $"Bearer {options.ApiToken}";
clientOptions.DefaultRequestHeaders["X-GitHub-Api-Version"] = "2022-11-28";
});
services.AddSingleton<GhsaDiagnostics>();
services.AddTransient<GhsaConnector>();
return services;
}
}

View File

@@ -0,0 +1,135 @@
using System.Collections.Generic;
using System.Linq;
using MongoDB.Bson;
namespace StellaOps.Feedser.Source.Ghsa.Internal;
internal sealed record GhsaCursor(
DateTimeOffset? LastUpdatedExclusive,
DateTimeOffset? CurrentWindowStart,
DateTimeOffset? CurrentWindowEnd,
int NextPage,
IReadOnlyCollection<Guid> PendingDocuments,
IReadOnlyCollection<Guid> PendingMappings)
{
private static readonly IReadOnlyCollection<Guid> EmptyGuidList = Array.Empty<Guid>();
public static GhsaCursor Empty { get; } = new(
null,
null,
null,
1,
EmptyGuidList,
EmptyGuidList);
public BsonDocument ToBsonDocument()
{
var document = new BsonDocument
{
["nextPage"] = NextPage,
["pendingDocuments"] = new BsonArray(PendingDocuments.Select(id => id.ToString())),
["pendingMappings"] = new BsonArray(PendingMappings.Select(id => id.ToString())),
};
if (LastUpdatedExclusive.HasValue)
{
document["lastUpdatedExclusive"] = LastUpdatedExclusive.Value.UtcDateTime;
}
if (CurrentWindowStart.HasValue)
{
document["currentWindowStart"] = CurrentWindowStart.Value.UtcDateTime;
}
if (CurrentWindowEnd.HasValue)
{
document["currentWindowEnd"] = CurrentWindowEnd.Value.UtcDateTime;
}
return document;
}
public static GhsaCursor FromBson(BsonDocument? document)
{
if (document is null || document.ElementCount == 0)
{
return Empty;
}
var lastUpdatedExclusive = document.TryGetValue("lastUpdatedExclusive", out var lastUpdated)
? ParseDate(lastUpdated)
: null;
var windowStart = document.TryGetValue("currentWindowStart", out var windowStartValue)
? ParseDate(windowStartValue)
: null;
var windowEnd = document.TryGetValue("currentWindowEnd", out var windowEndValue)
? ParseDate(windowEndValue)
: null;
var nextPage = document.TryGetValue("nextPage", out var nextPageValue) && nextPageValue.IsInt32
? Math.Max(1, nextPageValue.AsInt32)
: 1;
var pendingDocuments = ReadGuidArray(document, "pendingDocuments");
var pendingMappings = ReadGuidArray(document, "pendingMappings");
return new GhsaCursor(
lastUpdatedExclusive,
windowStart,
windowEnd,
nextPage,
pendingDocuments,
pendingMappings);
}
public GhsaCursor WithPendingDocuments(IEnumerable<Guid> ids)
=> this with { PendingDocuments = ids?.Distinct().ToArray() ?? EmptyGuidList };
public GhsaCursor WithPendingMappings(IEnumerable<Guid> ids)
=> this with { PendingMappings = ids?.Distinct().ToArray() ?? EmptyGuidList };
public GhsaCursor WithLastUpdatedExclusive(DateTimeOffset? timestamp)
=> this with { LastUpdatedExclusive = timestamp };
public GhsaCursor WithCurrentWindowStart(DateTimeOffset? timestamp)
=> this with { CurrentWindowStart = timestamp };
public GhsaCursor WithCurrentWindowEnd(DateTimeOffset? timestamp)
=> this with { CurrentWindowEnd = timestamp };
public GhsaCursor WithNextPage(int page)
=> this with { NextPage = page < 1 ? 1 : page };
private static DateTimeOffset? ParseDate(BsonValue value)
{
return value.BsonType switch
{
BsonType.DateTime => DateTime.SpecifyKind(value.ToUniversalTime(), DateTimeKind.Utc),
BsonType.String when DateTimeOffset.TryParse(value.AsString, out var parsed) => parsed.ToUniversalTime(),
_ => null,
};
}
private static IReadOnlyCollection<Guid> ReadGuidArray(BsonDocument document, string field)
{
if (!document.TryGetValue(field, out var value) || value is not BsonArray array)
{
return EmptyGuidList;
}
var results = new List<Guid>(array.Count);
foreach (var element in array)
{
if (element is null)
{
continue;
}
if (Guid.TryParse(element.ToString(), out var guid))
{
results.Add(guid);
}
}
return results;
}
}

View File

@@ -0,0 +1,50 @@
using System.Diagnostics.Metrics;
namespace StellaOps.Feedser.Source.Ghsa.Internal;
public sealed class GhsaDiagnostics : IDisposable
{
private const string MeterName = "StellaOps.Feedser.Source.Ghsa";
private const string MeterVersion = "1.0.0";
private readonly Meter _meter;
private readonly Counter<long> _fetchAttempts;
private readonly Counter<long> _fetchDocuments;
private readonly Counter<long> _fetchFailures;
private readonly Counter<long> _fetchUnchanged;
private readonly Counter<long> _parseSuccess;
private readonly Counter<long> _parseFailures;
private readonly Counter<long> _parseQuarantine;
private readonly Counter<long> _mapSuccess;
public GhsaDiagnostics()
{
_meter = new Meter(MeterName, MeterVersion);
_fetchAttempts = _meter.CreateCounter<long>("ghsa.fetch.attempts", unit: "operations");
_fetchDocuments = _meter.CreateCounter<long>("ghsa.fetch.documents", unit: "documents");
_fetchFailures = _meter.CreateCounter<long>("ghsa.fetch.failures", unit: "operations");
_fetchUnchanged = _meter.CreateCounter<long>("ghsa.fetch.unchanged", unit: "operations");
_parseSuccess = _meter.CreateCounter<long>("ghsa.parse.success", unit: "documents");
_parseFailures = _meter.CreateCounter<long>("ghsa.parse.failures", unit: "documents");
_parseQuarantine = _meter.CreateCounter<long>("ghsa.parse.quarantine", unit: "documents");
_mapSuccess = _meter.CreateCounter<long>("ghsa.map.success", unit: "advisories");
}
public void FetchAttempt() => _fetchAttempts.Add(1);
public void FetchDocument() => _fetchDocuments.Add(1);
public void FetchFailure() => _fetchFailures.Add(1);
public void FetchUnchanged() => _fetchUnchanged.Add(1);
public void ParseSuccess() => _parseSuccess.Add(1);
public void ParseFailure() => _parseFailures.Add(1);
public void ParseQuarantine() => _parseQuarantine.Add(1);
public void MapSuccess(long count) => _mapSuccess.Add(count);
public void Dispose() => _meter.Dispose();
}

View File

@@ -0,0 +1,115 @@
using System.Collections.Generic;
using System.Globalization;
using System.Text.Json;
namespace StellaOps.Feedser.Source.Ghsa.Internal;
internal static class GhsaListParser
{
public static GhsaListPage Parse(ReadOnlySpan<byte> content, int currentPage, int pageSize)
{
using var document = JsonDocument.Parse(content.ToArray());
var root = document.RootElement;
var items = new List<GhsaListItem>();
DateTimeOffset? maxUpdated = null;
if (root.TryGetProperty("advisories", out var advisories) && advisories.ValueKind == JsonValueKind.Array)
{
foreach (var advisory in advisories.EnumerateArray())
{
if (advisory.ValueKind != JsonValueKind.Object)
{
continue;
}
var id = GetString(advisory, "ghsa_id");
if (string.IsNullOrWhiteSpace(id))
{
continue;
}
var updated = GetDate(advisory, "updated_at");
if (updated.HasValue && (!maxUpdated.HasValue || updated > maxUpdated))
{
maxUpdated = updated;
}
items.Add(new GhsaListItem(id, updated));
}
}
var hasMorePages = TryDetermineHasMore(root, currentPage, pageSize, items.Count, out var nextPage);
return new GhsaListPage(items, maxUpdated, hasMorePages, nextPage ?? currentPage + 1);
}
private static bool TryDetermineHasMore(JsonElement root, int currentPage, int pageSize, int itemCount, out int? nextPage)
{
nextPage = null;
if (root.TryGetProperty("pagination", out var pagination) && pagination.ValueKind == JsonValueKind.Object)
{
var hasNextPage = pagination.TryGetProperty("has_next_page", out var hasNext) && hasNext.ValueKind == JsonValueKind.True;
if (hasNextPage)
{
nextPage = currentPage + 1;
return true;
}
if (pagination.TryGetProperty("total_pages", out var totalPagesElement) && totalPagesElement.ValueKind == JsonValueKind.Number && totalPagesElement.TryGetInt32(out var totalPages))
{
if (currentPage < totalPages)
{
nextPage = currentPage + 1;
return true;
}
}
return false;
}
if (itemCount >= pageSize)
{
nextPage = currentPage + 1;
return true;
}
return false;
}
private static string? GetString(JsonElement element, string propertyName)
{
if (!element.TryGetProperty(propertyName, out var property))
{
return null;
}
return property.ValueKind switch
{
JsonValueKind.String => property.GetString(),
_ => null,
};
}
private static DateTimeOffset? GetDate(JsonElement element, string propertyName)
{
var value = GetString(element, propertyName);
if (string.IsNullOrWhiteSpace(value))
{
return null;
}
return DateTimeOffset.TryParse(value, CultureInfo.InvariantCulture, DateTimeStyles.AssumeUniversal, out var parsed)
? parsed.ToUniversalTime()
: null;
}
}
internal sealed record GhsaListPage(
IReadOnlyList<GhsaListItem> Items,
DateTimeOffset? MaxUpdated,
bool HasMorePages,
int NextPageCandidate);
internal sealed record GhsaListItem(string GhsaId, DateTimeOffset? UpdatedAt);

View File

@@ -0,0 +1,209 @@
using System.Collections.Generic;
using System.Linq;
using StellaOps.Feedser.Models;
using StellaOps.Feedser.Storage.Mongo.Documents;
namespace StellaOps.Feedser.Source.Ghsa.Internal;
internal static class GhsaMapper
{
private static readonly HashSet<string> SemVerEcosystems = new(StringComparer.OrdinalIgnoreCase)
{
"npm",
"maven",
"pip",
"rubygems",
"composer",
"nuget",
"go",
"cargo",
};
public static Advisory Map(GhsaRecordDto dto, DocumentRecord document, DateTimeOffset recordedAt)
{
ArgumentNullException.ThrowIfNull(dto);
ArgumentNullException.ThrowIfNull(document);
var fetchProvenance = new AdvisoryProvenance(
GhsaConnectorPlugin.SourceName,
"document",
document.Uri,
document.FetchedAt,
new[] { ProvenanceFieldMasks.Advisory });
var mapProvenance = new AdvisoryProvenance(
GhsaConnectorPlugin.SourceName,
"mapping",
dto.GhsaId,
recordedAt,
new[] { ProvenanceFieldMasks.Advisory });
var aliases = dto.Aliases
.Where(static alias => !string.IsNullOrWhiteSpace(alias))
.Distinct(StringComparer.OrdinalIgnoreCase)
.ToArray();
var references = dto.References
.Select(reference => CreateReference(reference, recordedAt))
.Where(static reference => reference is not null)
.Cast<AdvisoryReference>()
.ToList();
var affected = CreateAffectedPackages(dto, recordedAt);
var credits = CreateCredits(dto.Credits, recordedAt);
var severity = dto.Severity?.ToLowerInvariant();
var summary = dto.Summary ?? dto.Description;
return new Advisory(
advisoryKey: dto.GhsaId,
title: dto.Summary ?? dto.GhsaId,
summary: summary,
language: "en",
published: dto.PublishedAt,
modified: dto.UpdatedAt ?? dto.PublishedAt,
severity: severity,
exploitKnown: false,
aliases: aliases,
credits: credits,
references: references,
affectedPackages: affected,
cvssMetrics: Array.Empty<CvssMetric>(),
provenance: new[] { fetchProvenance, mapProvenance });
}
private static AdvisoryReference? CreateReference(GhsaReferenceDto reference, DateTimeOffset recordedAt)
{
if (string.IsNullOrWhiteSpace(reference.Url) || !Validation.LooksLikeHttpUrl(reference.Url))
{
return null;
}
var kind = reference.Type?.ToLowerInvariant();
return new AdvisoryReference(
reference.Url,
kind,
reference.Name,
summary: null,
provenance: new AdvisoryProvenance(
GhsaConnectorPlugin.SourceName,
"reference",
reference.Url,
recordedAt,
new[] { ProvenanceFieldMasks.References }));
}
private static IReadOnlyList<AffectedPackage> CreateAffectedPackages(GhsaRecordDto dto, DateTimeOffset recordedAt)
{
if (dto.Affected.Count == 0)
{
return Array.Empty<AffectedPackage>();
}
var packages = new List<AffectedPackage>(dto.Affected.Count);
foreach (var affected in dto.Affected)
{
var ecosystem = string.IsNullOrWhiteSpace(affected.Ecosystem) ? "unknown" : affected.Ecosystem.Trim();
var packageName = string.IsNullOrWhiteSpace(affected.PackageName) ? "unknown-package" : affected.PackageName.Trim();
var identifier = $"{ecosystem.ToLowerInvariant()}:{packageName}";
var provenance = new[]
{
new AdvisoryProvenance(
GhsaConnectorPlugin.SourceName,
"affected",
identifier,
recordedAt,
new[] { ProvenanceFieldMasks.AffectedPackages }),
};
var rangeKind = SemVerEcosystems.Contains(ecosystem) ? "semver" : "vendor";
var packageType = SemVerEcosystems.Contains(ecosystem) ? AffectedPackageTypes.SemVer : AffectedPackageTypes.Vendor;
var versionRanges = new List<AffectedVersionRange>();
if (!string.IsNullOrWhiteSpace(affected.VulnerableRange) || !string.IsNullOrWhiteSpace(affected.PatchedVersion))
{
var primitives = new RangePrimitives(null, null, null, new Dictionary<string, string>(StringComparer.OrdinalIgnoreCase)
{
["ecosystem"] = ecosystem,
["package"] = packageName,
});
versionRanges.Add(new AffectedVersionRange(
rangeKind,
introducedVersion: null,
fixedVersion: Validation.TrimToNull(affected.PatchedVersion),
lastAffectedVersion: null,
rangeExpression: Validation.TrimToNull(affected.VulnerableRange),
provenance: new AdvisoryProvenance(
GhsaConnectorPlugin.SourceName,
"affected-range",
identifier,
recordedAt,
new[] { ProvenanceFieldMasks.VersionRanges }),
primitives: primitives));
}
var statuses = new[]
{
new AffectedPackageStatus(
"affected",
new AdvisoryProvenance(
GhsaConnectorPlugin.SourceName,
"affected-status",
identifier,
recordedAt,
new[] { ProvenanceFieldMasks.PackageStatuses })),
};
packages.Add(new AffectedPackage(
packageType,
identifier,
platform: null,
versionRanges: versionRanges,
statuses: statuses,
provenance: provenance));
}
return packages;
}
private static IReadOnlyList<AdvisoryCredit> CreateCredits(IReadOnlyList<GhsaCreditDto> credits, DateTimeOffset recordedAt)
{
if (credits.Count == 0)
{
return Array.Empty<AdvisoryCredit>();
}
var results = new List<AdvisoryCredit>(credits.Count);
foreach (var credit in credits)
{
var displayName = Validation.TrimToNull(credit.Name) ?? Validation.TrimToNull(credit.Login);
if (displayName is null)
{
continue;
}
var contacts = new List<string>();
if (!string.IsNullOrWhiteSpace(credit.ProfileUrl) && Validation.LooksLikeHttpUrl(credit.ProfileUrl))
{
contacts.Add(credit.ProfileUrl.Trim());
}
else if (!string.IsNullOrWhiteSpace(credit.Login))
{
contacts.Add($"https://github.com/{credit.Login.Trim()}");
}
var provenance = new AdvisoryProvenance(
GhsaConnectorPlugin.SourceName,
"credit",
displayName,
recordedAt,
new[] { ProvenanceFieldMasks.Credits });
results.Add(new AdvisoryCredit(displayName, credit.Type, contacts, provenance));
}
return results.Count == 0 ? Array.Empty<AdvisoryCredit>() : results;
}
}

View File

@@ -0,0 +1,55 @@
namespace StellaOps.Feedser.Source.Ghsa.Internal;
internal sealed record GhsaRecordDto
{
public string GhsaId { get; init; } = string.Empty;
public string? Summary { get; init; }
public string? Description { get; init; }
public string? Severity { get; init; }
public DateTimeOffset? PublishedAt { get; init; }
public DateTimeOffset? UpdatedAt { get; init; }
public IReadOnlyList<string> Aliases { get; init; } = Array.Empty<string>();
public IReadOnlyList<GhsaReferenceDto> References { get; init; } = Array.Empty<GhsaReferenceDto>();
public IReadOnlyList<GhsaAffectedDto> Affected { get; init; } = Array.Empty<GhsaAffectedDto>();
public IReadOnlyList<GhsaCreditDto> Credits { get; init; } = Array.Empty<GhsaCreditDto>();
}
internal sealed record GhsaReferenceDto
{
public string Url { get; init; } = string.Empty;
public string? Type { get; init; }
public string? Name { get; init; }
}
internal sealed record GhsaAffectedDto
{
public string PackageName { get; init; } = string.Empty;
public string Ecosystem { get; init; } = string.Empty;
public string? VulnerableRange { get; init; }
public string? PatchedVersion { get; init; }
}
internal sealed record GhsaCreditDto
{
public string? Type { get; init; }
public string? Name { get; init; }
public string? Login { get; init; }
public string? ProfileUrl { get; init; }
}

View File

@@ -0,0 +1,205 @@
using System.Collections.Generic;
using System.Globalization;
using System.Text.Json;
namespace StellaOps.Feedser.Source.Ghsa.Internal;
internal static class GhsaRecordParser
{
public static GhsaRecordDto Parse(ReadOnlySpan<byte> content)
{
using var document = JsonDocument.Parse(content.ToArray());
var root = document.RootElement;
var ghsaId = GetString(root, "ghsa_id") ?? throw new JsonException("ghsa_id missing");
var summary = GetString(root, "summary");
var description = GetString(root, "description");
var severity = GetString(root, "severity");
var publishedAt = GetDate(root, "published_at");
var updatedAt = GetDate(root, "updated_at") ?? publishedAt;
var aliases = new HashSet<string>(StringComparer.OrdinalIgnoreCase)
{
ghsaId,
};
if (root.TryGetProperty("cve_ids", out var cveIds) && cveIds.ValueKind == JsonValueKind.Array)
{
foreach (var cve in cveIds.EnumerateArray())
{
if (cve.ValueKind == JsonValueKind.String && !string.IsNullOrWhiteSpace(cve.GetString()))
{
aliases.Add(cve.GetString()!);
}
}
}
var references = ParseReferences(root);
var affected = ParseAffected(root);
var credits = ParseCredits(root);
return new GhsaRecordDto
{
GhsaId = ghsaId,
Summary = summary,
Description = description,
Severity = severity,
PublishedAt = publishedAt,
UpdatedAt = updatedAt,
Aliases = aliases.ToArray(),
References = references,
Affected = affected,
Credits = credits,
};
}
private static IReadOnlyList<GhsaReferenceDto> ParseReferences(JsonElement root)
{
if (!root.TryGetProperty("references", out var references) || references.ValueKind != JsonValueKind.Array)
{
return Array.Empty<GhsaReferenceDto>();
}
var list = new List<GhsaReferenceDto>(references.GetArrayLength());
foreach (var reference in references.EnumerateArray())
{
if (reference.ValueKind != JsonValueKind.Object)
{
continue;
}
var url = GetString(reference, "url");
if (string.IsNullOrWhiteSpace(url))
{
continue;
}
list.Add(new GhsaReferenceDto
{
Url = url,
Type = GetString(reference, "type"),
Name = GetString(reference, "name"),
});
}
return list;
}
private static IReadOnlyList<GhsaAffectedDto> ParseAffected(JsonElement root)
{
if (!root.TryGetProperty("vulnerabilities", out var vulnerabilities) || vulnerabilities.ValueKind != JsonValueKind.Array)
{
return Array.Empty<GhsaAffectedDto>();
}
var list = new List<GhsaAffectedDto>(vulnerabilities.GetArrayLength());
foreach (var entry in vulnerabilities.EnumerateArray())
{
if (entry.ValueKind != JsonValueKind.Object)
{
continue;
}
var package = entry.TryGetProperty("package", out var packageElement) && packageElement.ValueKind == JsonValueKind.Object
? packageElement
: default;
var packageName = GetString(package, "name") ?? "unknown-package";
var ecosystem = GetString(package, "ecosystem") ?? "unknown";
var vulnerableRange = GetString(entry, "vulnerable_version_range");
string? patchedVersion = null;
if (entry.TryGetProperty("first_patched_version", out var patchedElement) && patchedElement.ValueKind == JsonValueKind.Object)
{
patchedVersion = GetString(patchedElement, "identifier");
}
list.Add(new GhsaAffectedDto
{
PackageName = packageName,
Ecosystem = ecosystem,
VulnerableRange = vulnerableRange,
PatchedVersion = patchedVersion,
});
}
return list;
}
private static IReadOnlyList<GhsaCreditDto> ParseCredits(JsonElement root)
{
if (!root.TryGetProperty("credits", out var credits) || credits.ValueKind != JsonValueKind.Array)
{
return Array.Empty<GhsaCreditDto>();
}
var list = new List<GhsaCreditDto>(credits.GetArrayLength());
foreach (var credit in credits.EnumerateArray())
{
if (credit.ValueKind != JsonValueKind.Object)
{
continue;
}
var type = GetString(credit, "type");
var name = GetString(credit, "name");
string? login = null;
string? profile = null;
if (credit.TryGetProperty("user", out var user) && user.ValueKind == JsonValueKind.Object)
{
login = GetString(user, "login");
profile = GetString(user, "html_url") ?? GetString(user, "url");
name ??= GetString(user, "name");
}
name ??= login;
if (string.IsNullOrWhiteSpace(name))
{
continue;
}
list.Add(new GhsaCreditDto
{
Type = type,
Name = name,
Login = login,
ProfileUrl = profile,
});
}
return list;
}
private static string? GetString(JsonElement element, string propertyName)
{
if (element.ValueKind != JsonValueKind.Object)
{
return null;
}
if (!element.TryGetProperty(propertyName, out var property))
{
return null;
}
return property.ValueKind switch
{
JsonValueKind.String => property.GetString(),
_ => null,
};
}
private static DateTimeOffset? GetDate(JsonElement element, string propertyName)
{
var value = GetString(element, propertyName);
if (string.IsNullOrWhiteSpace(value))
{
return null;
}
return DateTimeOffset.TryParse(value, CultureInfo.InvariantCulture, DateTimeStyles.AssumeUniversal, out var parsed)
? parsed.ToUniversalTime()
: null;
}
}

View File

@@ -0,0 +1,43 @@
using StellaOps.Feedser.Core.Jobs;
namespace StellaOps.Feedser.Source.Ghsa;
internal static class GhsaJobKinds
{
public const string Fetch = "source:ghsa:fetch";
public const string Parse = "source:ghsa:parse";
public const string Map = "source:ghsa:map";
}
internal sealed class GhsaFetchJob : IJob
{
private readonly GhsaConnector _connector;
public GhsaFetchJob(GhsaConnector connector)
=> _connector = connector ?? throw new ArgumentNullException(nameof(connector));
public Task ExecuteAsync(JobExecutionContext context, CancellationToken cancellationToken)
=> _connector.FetchAsync(context.Services, cancellationToken);
}
internal sealed class GhsaParseJob : IJob
{
private readonly GhsaConnector _connector;
public GhsaParseJob(GhsaConnector connector)
=> _connector = connector ?? throw new ArgumentNullException(nameof(connector));
public Task ExecuteAsync(JobExecutionContext context, CancellationToken cancellationToken)
=> _connector.ParseAsync(context.Services, cancellationToken);
}
internal sealed class GhsaMapJob : IJob
{
private readonly GhsaConnector _connector;
public GhsaMapJob(GhsaConnector connector)
=> _connector = connector ?? throw new ArgumentNullException(nameof(connector));
public Task ExecuteAsync(JobExecutionContext context, CancellationToken cancellationToken)
=> _connector.MapAsync(context.Services, cancellationToken);
}

View File

@@ -0,0 +1,3 @@
using System.Runtime.CompilerServices;
[assembly: InternalsVisibleTo("FixtureUpdater")]

View File

@@ -0,0 +1,16 @@
<Project Sdk="Microsoft.NET.Sdk">
<PropertyGroup>
<TargetFramework>net10.0</TargetFramework>
<ImplicitUsings>enable</ImplicitUsings>
<Nullable>enable</Nullable>
</PropertyGroup>
<ItemGroup>
<ProjectReference Include="../StellaOps.Plugin/StellaOps.Plugin.csproj" />
<ProjectReference Include="../StellaOps.Feedser.Source.Common/StellaOps.Feedser.Source.Common.csproj" />
<ProjectReference Include="../StellaOps.Feedser.Models/StellaOps.Feedser.Models.csproj" />
</ItemGroup>
</Project>

View File

@@ -0,0 +1,13 @@
# TASKS
| Task | Owner(s) | Depends on | Notes |
|---|---|---|---|
|Select GHSA data source & auth model|BE-Conn-GHSA|Research|**DONE (2025-10-10)** Adopted GitHub Security Advisories REST (global) endpoint with bearer token + API version headers documented in `GhsaOptions`.|
|Fetch pipeline & state management|BE-Conn-GHSA|Source.Common, Storage.Mongo|**DONE (2025-10-10)** Implemented list/detail fetch using `GhsaCursor` (time window + page), resumable SourceState and backoff controls.|
|DTO & parser implementation|BE-Conn-GHSA|Source.Common|**DONE (2025-10-10)** Added `GhsaRecordParser`/DTOs extracting aliases, references, severity, vulnerable ranges, patched versions.|
|Canonical mapping & range primitives|BE-Conn-GHSA|Models|**DONE (2025-10-10)** `GhsaMapper` emits GHSA advisories with SemVer packages, vendor extensions (ecosystem/package) and deterministic references.|
|Deterministic fixtures & tests|QA|Testing|**DONE (2025-10-10)** New `StellaOps.Feedser.Source.Ghsa.Tests` regression covers fetch/parse/map via canned GHSA fixtures and snapshot assertions.|
|Telemetry & documentation|DevEx|Docs|**DONE (2025-10-10)** Diagnostics meter (`ghsa.fetch.*`) wired; DI extension documents token/headers and job registrations.|
|GitHub quota monitoring & retries|BE-Conn-GHSA, Observability|Source.Common|**TODO** Add rate-limit dashboard/alerts consuming `X-RateLimit-*` headers, tune retry/backoff strategy for 403/secondary rate limits, and document mitigation steps.|
|Production credential & scheduler rollout|Ops, BE-Conn-GHSA|Docs, WebService|**TODO** Issue PAT for production, update Helm/compose secrets, register fetch/parse/map cron defaults, and schedule staged backfill with health checks.|
|FEEDCONN-GHSA-02-004 GHSA credits & ecosystem severity mapping|BE-Conn-GHSA|Models `FEEDMODELS-SCHEMA-01-002`|**DONE (2025-10-11)** Mapper emits advisory credits with provenance masks, fixtures assert role/contact ordering, and severity normalization remains unchanged.|
|FEEDCONN-GHSA-02-007 Credit parity regression fixtures|BE-Conn-GHSA, QA|Source.Nvd, Source.Osv|**TODO** Capture GHSA/NVD/OSV triples with acknowledgements, wire conflict tests, and document fixture refresh workflow in `docs/dev/fixtures.md`.|