From 15be8657eaccc4d68924ceb68f531ee433dfc7b2 Mon Sep 17 00:00:00 2001 From: Niklas Date: Tue, 28 Jan 2025 16:07:27 +0100 Subject: [PATCH] Handle GitHub GraphQL API rate limiting (#1649) --- mirror-service/pom.xml | 5 + .../github/GitHubAdvisoryToCdxParser.java | 76 +++++-- .../github/GitHubApiClientFactory.java | 26 ++- .../GitHubHttpRequestRetryStrategy.java | 175 +++++++++++++++ .../datasource/github/GitHubMirror.java | 49 ++++- .../github/GitHubReflectionConfiguration.java | 1 + .../vulnmirror/KafkaStreamsTopologyIT.java | 4 +- .../GitHubHttpRequestRetryStrategyTest.java | 200 ++++++++++++++++++ .../datasource/github/advisories-page-01.json | 16 +- .../datasource/github/advisories-page-02.json | 8 +- .../datasource/github/advisory-02.json | 8 +- .../datasource/github/advisory-withdrawn.json | 8 +- .../resources/datasource/github/advisory.json | 8 +- pom.xml | 7 + 14 files changed, 531 insertions(+), 60 deletions(-) create mode 100644 mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubHttpRequestRetryStrategy.java create mode 100644 mirror-service/src/test/java/org/dependencytrack/vulnmirror/datasource/github/GitHubHttpRequestRetryStrategyTest.java diff --git a/mirror-service/pom.xml b/mirror-service/pom.xml index cd73f8cf0..63a2e5379 100644 --- a/mirror-service/pom.xml +++ b/mirror-service/pom.xml @@ -98,6 +98,11 @@ cvss-calculator + + org.apache.httpcomponents.client5 + httpclient5 + + com.github.package-url packageurl-java diff --git a/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubAdvisoryToCdxParser.java b/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubAdvisoryToCdxParser.java index c340883dc..769d46967 100644 --- a/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubAdvisoryToCdxParser.java +++ b/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubAdvisoryToCdxParser.java @@ -25,6 +25,7 @@ import com.google.protobuf.util.Timestamps; import io.github.jeremylong.openvulnerability.client.ghsa.CWEs; import io.github.jeremylong.openvulnerability.client.ghsa.Identifier; +import io.github.jeremylong.openvulnerability.client.ghsa.Package; import io.github.jeremylong.openvulnerability.client.ghsa.SecurityAdvisory; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -58,9 +59,9 @@ import java.util.HashMap; import java.util.List; import java.util.Optional; -import java.util.Set; import java.util.UUID; +import static com.github.packageurl.PackageURLBuilder.aPackageURL; import static io.github.nscuro.versatile.VersUtils.versFromGhsaRange; public class GitHubAdvisoryToCdxParser { @@ -106,7 +107,7 @@ public static Bom parse(final SecurityAdvisory advisory, boolean aliasSyncEnable CollectionUtils.isNotEmpty(advisory.getVulnerabilities().getEdges())) { for (final io.github.jeremylong.openvulnerability.client.ghsa.Vulnerability gitHubVulnerability : advisory.getVulnerabilities().getEdges()) { - PackageURL purl = generatePurlFromGitHubVulnerability(gitHubVulnerability); + PackageURL purl = convertToPurl(gitHubVulnerability.getPackage()); if (purl == null) { //drop mapping if purl is null break; @@ -152,8 +153,10 @@ public static Bom parse(final SecurityAdvisory advisory, boolean aliasSyncEnable } private static Optional parseRating(final SecurityAdvisory advisory) { - if (advisory.getCvss() != null && StringUtils.trimToNull(advisory.getCvss().getVectorString()) != null) { - final String cvssVector = StringUtils.trimToNull(advisory.getCvss().getVectorString()); + if (advisory.getCvssSeverities() != null + && advisory.getCvssSeverities().getCvssV3() != null + && StringUtils.trimToNull(advisory.getCvssSeverities().getCvssV3().getVectorString()) != null) { + final String cvssVector = StringUtils.trimToNull(advisory.getCvssSeverities().getCvssV3().getVectorString()); final Cvss cvss; try { cvss = Cvss.fromVector(cvssVector); @@ -262,27 +265,54 @@ private static List parseCwes(CWEs weaknesses) { return cwes; } - private static PackageURL generatePurlFromGitHubVulnerability(final io.github.jeremylong.openvulnerability.client.ghsa.Vulnerability vuln) { - final String purlType = ParserUtil.mapGitHubEcosystemToPurlType(vuln.getPackage().getEcosystem()); - try { - if (purlType != null) { - if (PackageURL.StandardTypes.NPM.equals(purlType) && vuln.getPackage().getName().contains("/")) { - final String[] parts = vuln.getPackage().getName().split("/"); - return PackageURLBuilder.aPackageURL().withType(purlType).withNamespace(parts[0]).withName(parts[1]).build(); - } else if (PackageURL.StandardTypes.MAVEN.equals(purlType) && vuln.getPackage().getName().contains(":")) { - final String[] parts = vuln.getPackage().getName().split(":"); - return PackageURLBuilder.aPackageURL().withType(purlType).withNamespace(parts[0]).withName(parts[1]).build(); - } else if (Set.of(PackageURL.StandardTypes.COMPOSER, PackageURL.StandardTypes.GOLANG).contains(purlType) && vuln.getPackage().getName().contains("/")) { - final String[] parts = vuln.getPackage().getName().split("/"); - final String namespace = String.join("/", Arrays.copyOfRange(parts, 0, parts.length - 1)); - return PackageURLBuilder.aPackageURL().withType(purlType).withNamespace(namespace).withName(parts[parts.length - 1]).build(); - } else { - return PackageURLBuilder.aPackageURL().withType(purlType).withName(vuln.getPackage().getName()).build(); - } + private static PackageURL convertToPurl(final Package pkg) { + final String purlType = switch (pkg.getEcosystem().toLowerCase()) { + case "composer" -> PackageURL.StandardTypes.COMPOSER; + case "erlang" -> PackageURL.StandardTypes.HEX; + case "go" -> PackageURL.StandardTypes.GOLANG; + case "maven" -> PackageURL.StandardTypes.MAVEN; + case "npm" -> PackageURL.StandardTypes.NPM; + case "nuget" -> PackageURL.StandardTypes.NUGET; + case "other" -> PackageURL.StandardTypes.GENERIC; + case "pip" -> PackageURL.StandardTypes.PYPI; + case "pub" -> "pub"; // https://github.com/package-url/purl-spec/blob/master/PURL-TYPES.rst#pub + case "rubygems" -> PackageURL.StandardTypes.GEM; + case "rust" -> PackageURL.StandardTypes.CARGO; + case "swift" -> "swift"; // https://github.com/package-url/purl-spec/blob/master/PURL-TYPES.rst#swift + default -> { + // Not optimal, but still better than ignoring the package entirely. + LOGGER.warn("Unrecognized ecosystem %s; Assuming PURL type %s for %s".formatted( + pkg.getEcosystem(), PackageURL.StandardTypes.GENERIC, pkg)); + yield PackageURL.StandardTypes.GENERIC; } + }; + + final PackageURLBuilder purlBuilder = aPackageURL().withType(purlType); + if (PackageURL.StandardTypes.MAVEN.equals(purlType) && pkg.getName().contains(":")) { + final String[] nameParts = pkg.getName().split(":", 2); + purlBuilder + .withNamespace(nameParts[0]) + .withName(nameParts[1]); + } else if ((PackageURL.StandardTypes.COMPOSER.equals(purlType) + || PackageURL.StandardTypes.GOLANG.equals(purlType) + || PackageURL.StandardTypes.NPM.equals(purlType) + || PackageURL.StandardTypes.GENERIC.equals(purlType)) + && pkg.getName().contains("/")) { + final String[] nameParts = pkg.getName().split("/"); + final String namespace = String.join("/", Arrays.copyOfRange(nameParts, 0, nameParts.length - 1)); + purlBuilder + .withNamespace(namespace) + .withName(nameParts[nameParts.length - 1]); + } else { + purlBuilder.withName(pkg.getName()); + } + + try { + return purlBuilder.build(); } catch (MalformedPackageURLException e) { - LOGGER.warn("Unable to create purl from GitHub Vulnerability. Skipping " + vuln.getPackage().getEcosystem() + " : " + vuln.getPackage().getName()); + LOGGER.warn("Failed to assemble a valid PURL from {}", pkg, e); + return null; } - return null; } + } diff --git a/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubApiClientFactory.java b/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubApiClientFactory.java index b86de8c89..25e158b67 100644 --- a/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubApiClientFactory.java +++ b/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubApiClientFactory.java @@ -20,9 +20,11 @@ import io.github.jeremylong.openvulnerability.client.ghsa.GitHubSecurityAdvisoryClient; import io.github.jeremylong.openvulnerability.client.ghsa.GitHubSecurityAdvisoryClientBuilder; -import jakarta.enterprise.context.ApplicationScoped; +import org.apache.hc.client5.http.impl.async.HttpAsyncClientBuilder; +import org.apache.hc.client5.http.impl.async.HttpAsyncClients; import org.dependencytrack.common.SecretDecryptor; +import jakarta.enterprise.context.ApplicationScoped; import java.time.Instant; import java.time.ZoneOffset; import java.time.ZonedDateTime; @@ -41,17 +43,23 @@ class GitHubApiClientFactory { } GitHubSecurityAdvisoryClient create(final long lastUpdatedEpochSeconds) { - final GitHubSecurityAdvisoryClientBuilder builder = aGitHubSecurityAdvisoryClient(); + final HttpAsyncClientBuilder httpClientBuilder = HttpAsyncClients.custom() + .setRetryStrategy(new GitHubHttpRequestRetryStrategy()) + .useSystemProperties(); + + final GitHubSecurityAdvisoryClientBuilder builder = aGitHubSecurityAdvisoryClient() + .withHttpClientSupplier(httpClientBuilder::build); config.baseUrl().ifPresent(builder::withEndpoint); config.apiKey() - .map(encryptedApiKey -> { - try { - return secretDecryptor.decryptAsString(encryptedApiKey); - } catch (Exception e) { - throw new IllegalStateException("Failed to decrypt API key", e); - } - }) + // TODO: https://github.com/DependencyTrack/dependency-track/issues/3332 +// .map(encryptedApiKey -> { +// try { +// return secretDecryptor.decryptAsString(encryptedApiKey); +// } catch (Exception e) { +// throw new IllegalStateException("Failed to decrypt API key", e); +// } +// }) .ifPresent(builder::withApiKey); if (lastUpdatedEpochSeconds > 0) { diff --git a/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubHttpRequestRetryStrategy.java b/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubHttpRequestRetryStrategy.java new file mode 100644 index 000000000..8cdafc9c6 --- /dev/null +++ b/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubHttpRequestRetryStrategy.java @@ -0,0 +1,175 @@ +/* + * This file is part of Dependency-Track. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * SPDX-License-Identifier: Apache-2.0 + * Copyright (c) OWASP Foundation. All Rights Reserved. + */ +package org.dependencytrack.vulnmirror.datasource.github; + +import org.apache.hc.client5.http.impl.DefaultHttpRequestRetryStrategy; +import org.apache.hc.core5.http.ConnectionClosedException; +import org.apache.hc.core5.http.Header; +import org.apache.hc.core5.http.HttpResponse; +import org.apache.hc.core5.http.protocol.HttpContext; +import org.apache.hc.core5.util.TimeValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLException; +import java.io.InterruptedIOException; +import java.net.ConnectException; +import java.net.NoRouteToHostException; +import java.net.UnknownHostException; +import java.time.Duration; +import java.time.Instant; +import java.util.List; + +final class GitHubHttpRequestRetryStrategy extends DefaultHttpRequestRetryStrategy { + + private enum RateLimitStrategy { + RETRY_AFTER, + LIMIT_RESET + } + + private record RateLimitInfo( + RateLimitStrategy strategy, + Duration retryAfter, + Long remainingRequests, + Long requestLimit, + Instant requestLimitResetAt) { + + private static RateLimitInfo of(final HttpResponse response) { + final Header retryAfterHeader = response.getFirstHeader("retry-after"); + if (retryAfterHeader != null) { + final long retryAfterSeconds = Long.parseLong(retryAfterHeader.getValue().trim()); + return new RateLimitInfo(RateLimitStrategy.RETRY_AFTER, Duration.ofSeconds(retryAfterSeconds), null, null, null); + } + + final Header remainingRequestsHeader = response.getFirstHeader("x-ratelimit-remaining"); + if (remainingRequestsHeader != null) { + final long remainingRequests = Long.parseLong(remainingRequestsHeader.getValue().trim()); + final long requestLimit = Long.parseLong(response.getFirstHeader("x-ratelimit-limit").getValue().trim()); + final long requestLimitResetEpochSeconds = Long.parseLong(response.getFirstHeader("x-ratelimit-reset").getValue().trim()); + return new RateLimitInfo(RateLimitStrategy.LIMIT_RESET, null, remainingRequests, requestLimit, Instant.ofEpochSecond(requestLimitResetEpochSeconds)); + } + + return null; + } + + } + + private static final Logger LOGGER = LoggerFactory.getLogger(GitHubHttpRequestRetryStrategy.class); + + private final Duration maxRetryDelay = Duration.ofMinutes(3); + + GitHubHttpRequestRetryStrategy() { + super( + /* maxRetries */ 6, + /* defaultRetryInterval */ TimeValue.ofSeconds(1L), + // Same as DefaultHttpRequestRetryStrategy. + /* retryableExceptions */ List.of( + ConnectException.class, + ConnectionClosedException.class, + InterruptedIOException.class, + NoRouteToHostException.class, + SSLException.class, + UnknownHostException.class), + // Same as DefaultHttpRequestRetryStrategy, with addition of 403, + // since GitHub might use that status to indicate rate limiting. + /* retryableCodes */ List.of(403, 429, 503)); + } + + @Override + public boolean retryRequest(final HttpResponse response, final int execCount, final HttpContext context) { + if (response.getCode() != 403 && response.getCode() != 429) { + return super.retryRequest(response, execCount, context); + } + + final var rateLimitInfo = RateLimitInfo.of(response); + if (rateLimitInfo == null) { + if (response.getCode() == 403) { + // Authorization failure. Do not retry. + return false; + } + + return super.retryRequest(response, execCount, context); + } + + return switch (rateLimitInfo.strategy()) { + case RETRY_AFTER -> { + // Usually GitHub will request to wait for 1min. This may change though, and we can't risk + // blocking a worker thread unnecessarily for a long period of time. + if (rateLimitInfo.retryAfter().compareTo(maxRetryDelay) > 0) { + LOGGER.warn(""" + Rate limiting detected; GitHub API indicates retries to be acceptable after {}, \ + which exceeds the maximum retry duration of {}. \ + Not performing any further retries.""", + rateLimitInfo.retryAfter(), maxRetryDelay); + yield false; + } + + yield true; + } + case LIMIT_RESET -> { + if (rateLimitInfo.remainingRequests() > 0) { + // Still have requests budget remaining. Failure reason is not rate limiting. + yield super.retryRequest(response, execCount, context); + } + + // The duration after which the limit is reset is not defined in GitHub's API docs. + // Need to safeguard ourselves from blocking the worker thread for too long. + final var untilResetDuration = Duration.between(Instant.now(), rateLimitInfo.requestLimitResetAt()); + if (untilResetDuration.compareTo(maxRetryDelay) > 0) { + LOGGER.warn(""" + Primary rate limit of {} requests exhausted. The rate limit will reset at {} (in {}), \ + which exceeds the maximum retry duration of {}. Not performing any further retries.""", + rateLimitInfo.requestLimit(), rateLimitInfo.requestLimitResetAt(), untilResetDuration, maxRetryDelay); + yield false; + } + + yield true; + } + }; + } + + @Override + public TimeValue getRetryInterval(final HttpResponse response, final int execCount, final HttpContext context) { + // When this is called, retryRequest was already invoked to determine whether + // a retry should be performed. So we can skip the status code check here. + + final var rateLimitInfo = RateLimitInfo.of(response); + if (rateLimitInfo == null) { + return super.getRetryInterval(response, execCount, context); + } + + return switch (rateLimitInfo.strategy()) { + case RETRY_AFTER -> { + LOGGER.warn(""" + Rate limiting detected; GitHub indicates retries to be acceptable after {}; \ + Will wait and try again.""", rateLimitInfo.retryAfter()); + yield TimeValue.ofMilliseconds(rateLimitInfo.retryAfter().toMillis()); + } + case LIMIT_RESET -> { + final var retryAfter = Duration.between(Instant.now(), rateLimitInfo.requestLimitResetAt()); + LOGGER.warn(""" + Primary rate limit of {} requests exhausted. Limit will reset at {}; \ + Will wait for {} and try again.""", + rateLimitInfo.requestLimit(), rateLimitInfo.requestLimitResetAt(), retryAfter); + yield TimeValue.ofMilliseconds(retryAfter.toMillis()); + } + }; + } + +} diff --git a/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubMirror.java b/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubMirror.java index b81b1caab..969e74d70 100644 --- a/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubMirror.java +++ b/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubMirror.java @@ -21,7 +21,6 @@ import io.github.jeremylong.openvulnerability.client.ghsa.GitHubSecurityAdvisoryClient; import io.github.jeremylong.openvulnerability.client.ghsa.SecurityAdvisory; import io.micrometer.core.instrument.Timer; -import jakarta.enterprise.context.ApplicationScoped; import org.apache.kafka.clients.producer.Producer; import org.cyclonedx.proto.v1_6.Bom; import org.dependencytrack.vulnmirror.datasource.AbstractDatasourceMirror; @@ -31,10 +30,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import jakarta.enterprise.context.ApplicationScoped; import java.time.Duration; import java.time.Instant; -import java.time.chrono.ChronoZonedDateTime; -import java.util.Optional; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.temporal.ChronoUnit; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -102,6 +103,9 @@ void mirrorInternal() throws Exception { LOGGER.info("Mirroring GitHub Advisories that were modified since {}", Instant.ofEpochSecond(lastModified)); final Timer.Sample durationSample = Timer.start(); + int numMirrored = 0; + Instant lastStatusLog = Instant.now(); + ZonedDateTime committedLastModified = null; try (final GitHubSecurityAdvisoryClient apiClient = apiClientFactory.create(lastModified)) { while (apiClient.hasNext()) { @@ -110,17 +114,44 @@ void mirrorInternal() throws Exception { Bom bov = GitHubAdvisoryToCdxParser.parse(advisory, this.config.aliasSyncEnabled().orElse(false)); publishIfChanged(bov); } + + final int currentNumMirrored = ++numMirrored; + if (lastStatusLog.plus(5, ChronoUnit.SECONDS).isBefore(Instant.now())) { + final int currentMirroredPercentage = (currentNumMirrored * 100) / apiClient.getTotalAvailable(); + LOGGER.info("Mirrored {}/{} GHSAs ({}%); Last committed modification timestamp: {}", + currentNumMirrored, apiClient.getTotalAvailable(), currentMirroredPercentage, committedLastModified); + lastStatusLog = Instant.now(); + } } - } - // lastUpdated is null when nothing changed - Optional.ofNullable(apiClient.getLastUpdated()) - .map(ChronoZonedDateTime::toEpochSecond) - .ifPresent(epochSeconds -> updateState(new GitHubMirrorState(epochSeconds))); + committedLastModified = maybeCommitLastModified(apiClient.getLastUpdated()); + } } finally { final long durationNanos = durationSample.stop(durationTimer); - LOGGER.info("Mirroring of GitHub Advisories completed in {}", Duration.ofNanos(durationNanos)); + LOGGER.info(""" + Mirroring of {} GitHub Advisories completed in {} \ + (last committed modification timestamp: {})""", + numMirrored, Duration.ofNanos(durationNanos), committedLastModified); } } + private ZonedDateTime maybeCommitLastModified(final ZonedDateTime lastModifiedDateTime) { + if (lastModifiedDateTime == null) { + return null; + } + + final ZonedDateTime previous = getState() + .map(GitHubMirrorState::lastUpdatedEpochSeconds) + .map(lastModifiedEpochSeconds -> ZonedDateTime.ofInstant( + Instant.ofEpochSecond(lastModifiedEpochSeconds), ZoneOffset.UTC)) + .orElseGet(() -> ZonedDateTime.ofInstant(Instant.EPOCH, ZoneOffset.UTC)); + if (previous.isBefore(lastModifiedDateTime)) { + LOGGER.debug("Updating last captured modification date: {} -> {}", previous, lastModifiedDateTime); + updateState(new GitHubMirrorState(lastModifiedDateTime.toEpochSecond())); + return lastModifiedDateTime; + } + + return previous; + } + } diff --git a/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubReflectionConfiguration.java b/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubReflectionConfiguration.java index 1185ec23c..97a544836 100644 --- a/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubReflectionConfiguration.java +++ b/mirror-service/src/main/java/org/dependencytrack/vulnmirror/datasource/github/GitHubReflectionConfiguration.java @@ -29,6 +29,7 @@ classNames = { "io.github.jeremylong.openvulnerability.client.ghsa.AbstractPageable", "io.github.jeremylong.openvulnerability.client.ghsa.CVSS", + "io.github.jeremylong.openvulnerability.client.ghsa.CVSSSeverities", "io.github.jeremylong.openvulnerability.client.ghsa.CWE", "io.github.jeremylong.openvulnerability.client.ghsa.CWEs", "io.github.jeremylong.openvulnerability.client.ghsa.Identifier", diff --git a/mirror-service/src/test/java/org/dependencytrack/vulnmirror/KafkaStreamsTopologyIT.java b/mirror-service/src/test/java/org/dependencytrack/vulnmirror/KafkaStreamsTopologyIT.java index d30ca5dbe..3dfef84aa 100644 --- a/mirror-service/src/test/java/org/dependencytrack/vulnmirror/KafkaStreamsTopologyIT.java +++ b/mirror-service/src/test/java/org/dependencytrack/vulnmirror/KafkaStreamsTopologyIT.java @@ -193,7 +193,9 @@ public static class TestProfile implements QuarkusTestProfile { public Map getConfigOverrides() { return Map.ofEntries( Map.entry("dtrack.vuln-source.github.advisories.enabled", "true"), - Map.entry("dtrack.vuln-source.github.advisories.access.token", /* encrypted "foobar" */ "0fsdpawY5lis3lZK7BiJmjR5Nr1MjXlqa3vso9kLSEM="), + // TODO: Should be an encrypted, base64-encoded value. + // https://github.com/DependencyTrack/dependency-track/issues/3332 + Map.entry("dtrack.vuln-source.github.advisories.access.token", "foobar"), Map.entry("dtrack.vuln-source.github.advisories.base.url", "http://localhost:${quarkus.wiremock.devservices.port}"), Map.entry("secret.key.path", "src/test/resources/secret.key") ); diff --git a/mirror-service/src/test/java/org/dependencytrack/vulnmirror/datasource/github/GitHubHttpRequestRetryStrategyTest.java b/mirror-service/src/test/java/org/dependencytrack/vulnmirror/datasource/github/GitHubHttpRequestRetryStrategyTest.java new file mode 100644 index 000000000..4fa50035f --- /dev/null +++ b/mirror-service/src/test/java/org/dependencytrack/vulnmirror/datasource/github/GitHubHttpRequestRetryStrategyTest.java @@ -0,0 +1,200 @@ +/* + * This file is part of Dependency-Track. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * SPDX-License-Identifier: Apache-2.0 + * Copyright (c) OWASP Foundation. All Rights Reserved. + */ +package org.dependencytrack.vulnmirror.datasource.github; + +import org.apache.hc.client5.http.protocol.HttpClientContext; +import org.apache.hc.core5.http.message.BasicHttpResponse; +import org.apache.hc.core5.util.TimeValue; +import org.assertj.core.data.Offset; +import org.junit.jupiter.api.Test; + +import java.time.Instant; + +import static org.assertj.core.api.Assertions.assertThat; + +class GitHubHttpRequestRetryStrategyTest { + + @Test + void shouldNotRetryOnResponseWithCode403() { + final var httpResponse = new BasicHttpResponse(403); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + final boolean shouldRetry = retryStrategy.retryRequest(httpResponse, 1, httpContext); + assertThat(shouldRetry).isFalse(); + } + + @Test + void shouldRetryOnResponseWithCode429() { + final var httpResponse = new BasicHttpResponse(429); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + final boolean shouldRetry = retryStrategy.retryRequest(httpResponse, 1, httpContext); + assertThat(shouldRetry).isTrue(); + } + + @Test + void shouldRetryOnResponseWithCode503() { + final var httpResponse = new BasicHttpResponse(503); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + final boolean shouldRetry = retryStrategy.retryRequest(httpResponse, 1, httpContext); + assertThat(shouldRetry).isTrue(); + } + + @Test + void shouldRetryUpToSixAttempts() { + final var httpResponse = new BasicHttpResponse(503); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + + boolean shouldRetry = retryStrategy.retryRequest(httpResponse, 6, httpContext); + assertThat(shouldRetry).isTrue(); + + shouldRetry = retryStrategy.retryRequest(httpResponse, 7, httpContext); + assertThat(shouldRetry).isFalse(); + } + + @Test + void shouldRetryOnResponseWithCode403AndRetryAfterHeader() { + final var httpResponse = new BasicHttpResponse(403); + httpResponse.addHeader("retry-after", /* 1min */ 60); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + final boolean shouldRetry = retryStrategy.retryRequest(httpResponse, 1, httpContext); + assertThat(shouldRetry).isTrue(); + } + + @Test + void shouldRetryOnResponseWithCode429AndRetryAfterHeader() { + final var httpResponse = new BasicHttpResponse(429); + httpResponse.addHeader("retry-after", /* 1min */ 60); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + final boolean shouldRetry = retryStrategy.retryRequest(httpResponse, 1, httpContext); + assertThat(shouldRetry).isTrue(); + } + + @Test + void shouldNotRetryWhenRetryAfterExceedsMaxDelay() { + final var httpResponse = new BasicHttpResponse(403); + httpResponse.addHeader("retry-after", /* 3min */ 180); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + boolean shouldRetry = retryStrategy.retryRequest(httpResponse, 1, httpContext); + assertThat(shouldRetry).isTrue(); + + httpResponse.setHeader("retry-after", /* 3min 1sec */ 181); + shouldRetry = retryStrategy.retryRequest(httpResponse, 1, httpContext); + assertThat(shouldRetry).isFalse(); + } + + @Test + void shouldRetryOnResponseWithCode403AndRateLimitHeaders() { + final var httpResponse = new BasicHttpResponse(403); + httpResponse.addHeader("x-ratelimit-remaining", 6); + httpResponse.addHeader("x-ratelimit-limit", 666); + httpResponse.setHeader("x-ratelimit-reset", Instant.now().getEpochSecond()); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + final boolean shouldRetry = retryStrategy.retryRequest(httpResponse, 1, httpContext); + assertThat(shouldRetry).isTrue(); + } + + @Test + void shouldRetryOnResponseWithCode429AndRateLimitHeaders() { + final var httpResponse = new BasicHttpResponse(429); + httpResponse.addHeader("x-ratelimit-remaining", 6); + httpResponse.addHeader("x-ratelimit-limit", 666); + httpResponse.setHeader("x-ratelimit-reset", Instant.now().getEpochSecond()); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + final boolean shouldRetry = retryStrategy.retryRequest(httpResponse, 1, httpContext); + assertThat(shouldRetry).isTrue(); + } + + @Test + void shouldRetryWhenLimitResetIsShorterThanMaxDelay() { + final var httpResponse = new BasicHttpResponse(429); + httpResponse.addHeader("x-ratelimit-remaining", 0); + httpResponse.addHeader("x-ratelimit-limit", 666); + httpResponse.setHeader("x-ratelimit-reset", Instant.now().plusSeconds(/* 3min */ 180).getEpochSecond()); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + final boolean shouldRetry = retryStrategy.retryRequest(httpResponse, 1, httpContext); + assertThat(shouldRetry).isTrue(); + } + + @Test + void shouldNotRetryWhenLimitResetExceedsMaxDelay() { + final var httpResponse = new BasicHttpResponse(429); + httpResponse.addHeader("x-ratelimit-remaining", 0); + httpResponse.addHeader("x-ratelimit-limit", 666); + httpResponse.setHeader("x-ratelimit-reset", Instant.now().plusSeconds(/* 3min 1sec */ 181).getEpochSecond()); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + final boolean shouldRetry = retryStrategy.retryRequest(httpResponse, 1, httpContext); + assertThat(shouldRetry).isFalse(); + } + + @Test + void shouldUseRetryAfterHeaderForRetryDelay() { + final var httpResponse = new BasicHttpResponse(429); + httpResponse.addHeader("retry-after", /* 1min 6sec */ 66); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + final TimeValue retryDelay = retryStrategy.getRetryInterval(httpResponse, 1, httpContext); + assertThat(retryDelay.toSeconds()).isEqualTo(66); + } + + @Test + void shouldUseLimitResetHeaderForRetryDelay() { + final var httpResponse = new BasicHttpResponse(429); + httpResponse.addHeader("x-ratelimit-remaining", 0); + httpResponse.addHeader("x-ratelimit-limit", 666); + httpResponse.addHeader("x-ratelimit-reset", Instant.now().plusSeconds(66).getEpochSecond()); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + final TimeValue retryDelay = retryStrategy.getRetryInterval(httpResponse, 1, httpContext); + assertThat(retryDelay.toSeconds()).isCloseTo(66, Offset.offset(1L)); + } + + @Test + void shouldUseOneSecondAsDefaultRetryDelay() { + final var httpResponse = new BasicHttpResponse(503); + final var httpContext = HttpClientContext.create(); + + final var retryStrategy = new GitHubHttpRequestRetryStrategy(); + final TimeValue retryDelay = retryStrategy.getRetryInterval(httpResponse, 1, httpContext); + assertThat(retryDelay.toSeconds()).isEqualTo(1); + } + +} \ No newline at end of file diff --git a/mirror-service/src/test/resources/datasource/github/advisories-page-01.json b/mirror-service/src/test/resources/datasource/github/advisories-page-01.json index bbae32361..6ca7c0526 100644 --- a/mirror-service/src/test/resources/datasource/github/advisories-page-01.json +++ b/mirror-service/src/test/resources/datasource/github/advisories-page-01.json @@ -100,9 +100,11 @@ "endCursor": "Y3Vyc29yOnYyOpK5MjAxOS0wMi0yMlQyMTo1MzowNCswMTowMM0H6w==" } }, - "cvss": { - "score": 0.0, - "vectorString": null + "cvssSeverities": { + "cvssV3": { + "score": 0.0, + "vectorString": null + } }, "cwes": { "edges": [], @@ -192,9 +194,11 @@ "endCursor": "Y3Vyc29yOnYyOpK5MjAxOS0wMi0yMlQyMTo1MzoxMyswMTowMM0H7g==" } }, - "cvss": { - "score": 0.0, - "vectorString": null + "cvssSeverities": { + "cvssV3": { + "score": 0.0, + "vectorString": null + } }, "cwes": { "edges": [], diff --git a/mirror-service/src/test/resources/datasource/github/advisories-page-02.json b/mirror-service/src/test/resources/datasource/github/advisories-page-02.json index 651a20572..eaf598383 100644 --- a/mirror-service/src/test/resources/datasource/github/advisories-page-02.json +++ b/mirror-service/src/test/resources/datasource/github/advisories-page-02.json @@ -85,9 +85,11 @@ "endCursor": "Y3Vyc29yOnYyOpK5MjAyMi0wOS0wOVQyMTozNDo0MCswMjowMM0R1Q==" } }, - "cvss": { - "score": 5.4, - "vectorString": "CVSS:3.1/AV:N/AC:L/PR:N/UI:R/S:U/C:L/I:L/A:N" + "cvssSeverities": { + "cvssV3": { + "score": 5.4, + "vectorString": "CVSS:3.1/AV:N/AC:L/PR:N/UI:R/S:U/C:L/I:L/A:N" + } }, "cwes": { "edges": [ diff --git a/mirror-service/src/test/resources/datasource/github/advisory-02.json b/mirror-service/src/test/resources/datasource/github/advisory-02.json index 7daf535b6..8a582af82 100644 --- a/mirror-service/src/test/resources/datasource/github/advisory-02.json +++ b/mirror-service/src/test/resources/datasource/github/advisory-02.json @@ -75,9 +75,11 @@ "endCursor": "Y3Vyc29yOnYyOpK5MjAyMi0wOS0wOVQyMTozNDo0MCswMjowMM0R1Q==" } }, - "cvss": { - "score": 5.4, - "vectorString": "CVSS:3.1/AV:N/AC:L/PR:N/UI:R/S:U/C:L/I:L/A:N" + "cvssSeverities": { + "cvssV3": { + "score": 5.4, + "vectorString": "CVSS:3.1/AV:N/AC:L/PR:N/UI:R/S:U/C:L/I:L/A:N" + } }, "cwes": { "edges": [ diff --git a/mirror-service/src/test/resources/datasource/github/advisory-withdrawn.json b/mirror-service/src/test/resources/datasource/github/advisory-withdrawn.json index 9db06178a..32eba6ecf 100644 --- a/mirror-service/src/test/resources/datasource/github/advisory-withdrawn.json +++ b/mirror-service/src/test/resources/datasource/github/advisory-withdrawn.json @@ -35,9 +35,11 @@ "endCursor": "Y3Vyc29yOnYyOpK5MjAxOS0wMi0yMlQyMTo1MzowNCswMTowMM0H6w==" } }, - "cvss": { - "score": 0.0, - "vectorString": null + "cvssSeverities": { + "cvssV3": { + "score": 0.0, + "vectorString": null + } }, "cwes": { "edges": [], diff --git a/mirror-service/src/test/resources/datasource/github/advisory.json b/mirror-service/src/test/resources/datasource/github/advisory.json index 69b5fa93c..02e74db4f 100644 --- a/mirror-service/src/test/resources/datasource/github/advisory.json +++ b/mirror-service/src/test/resources/datasource/github/advisory.json @@ -90,9 +90,11 @@ "endCursor": "Y3Vyc29yOnYyOpK5MjAxOS0wMi0yMlQyMTo1MzowNCswMTowMM0H6w==" } }, - "cvss": { - "score": 0.0, - "vectorString": null + "cvssSeverities": { + "cvssV3": { + "score": 0.0, + "vectorString": null + } }, "cwes": { "edges": [], diff --git a/pom.xml b/pom.xml index 335e46f62..80f555e65 100644 --- a/pom.xml +++ b/pom.xml @@ -72,6 +72,7 @@ 1.4.3 3.3.2 2.1.2 + 5.4.1 5.1.0 4.1.0 3.7.1 @@ -166,6 +167,12 @@ ${lib.confluent-parallel-consumer.version} + + org.apache.httpcomponents.client5 + httpclient5 + ${lib.httpclient5.version} + + com.fasterxml.uuid java-uuid-generator