refactor(npm): separate lockfile 3 detector (#504)

This commit is contained in:
Justin Perez 2023-04-12 14:57:08 -07:00 коммит произвёл GitHub
Родитель cef34c553c
Коммит f4ea6f977d
Не найден ключ, соответствующий данной подписи
Идентификатор ключа GPG: 4AEE18F83AFDEB23
9 изменённых файлов: 695 добавлений и 618 удалений

Просмотреть файл

@ -13,7 +13,3 @@ The environment variable `PyPiMaxCacheEntries` is used to control the size of th
The default value is 128.
[1]: https://go.dev/ref/mod#go-mod-graph
## `CD_LOCKFILE_V3_ENABLED`
If the environment variable `CD_LOCKFILE_V3_ENABLED` is set to "true", this will enable the `NpmDetectorWithRoots` to use the experiementental `package-lock.json` `lockfileVersion` 3 logic. Otherwise, the `package-lock.json` file will be parsed with the existing logic, which is broken on `lockfileVersion` 3.

Просмотреть файл

@ -1,358 +1,44 @@
namespace Microsoft.ComponentDetection.Detectors.Npm;
using System;
using System.Collections.Generic;
using System.IO;
using System.Linq;
using System.Reactive.Linq;
using System.Threading.Tasks;
using Microsoft.ComponentDetection.Common;
using Microsoft.ComponentDetection.Common.Telemetry.Records;
using Microsoft.ComponentDetection.Contracts;
using Microsoft.ComponentDetection.Contracts.Internal;
using Microsoft.ComponentDetection.Contracts.TypedComponent;
using Microsoft.Extensions.Logging;
using Newtonsoft.Json;
using Newtonsoft.Json.Linq;
public class NpmComponentDetectorWithRoots : FileComponentDetector
public class NpmComponentDetectorWithRoots : NpmLockfileDetectorBase
{
private const string NpmRegistryHost = "registry.npmjs.org";
private readonly object lernaFilesLock = new object();
public const string LernaSearchPattern = "lerna.json";
/// <summary>
/// Gets or sets the logger for writing basic logging message to both console and file.
/// </summary>
private readonly IPathUtilityService pathUtilityService;
private readonly IEnvironmentVariableService environmentVariableService;
public NpmComponentDetectorWithRoots(
IComponentStreamEnumerableFactory componentStreamEnumerableFactory,
IObservableDirectoryWalkerFactory walkerFactory,
IPathUtilityService pathUtilityService,
IEnvironmentVariableService environmentVariableService,
ILogger<NpmComponentDetectorWithRoots> logger)
: base(
componentStreamEnumerableFactory,
walkerFactory,
pathUtilityService,
logger)
{
this.ComponentStreamEnumerableFactory = componentStreamEnumerableFactory;
this.Scanner = walkerFactory;
this.pathUtilityService = pathUtilityService;
this.environmentVariableService = environmentVariableService;
this.Logger = logger;
}
public NpmComponentDetectorWithRoots(IPathUtilityService pathUtilityService) => this.pathUtilityService = pathUtilityService;
/// <summary>Common delegate for Package.json JToken processing.</summary>
/// <param name="token">A JToken, usually corresponding to a package.json file.</param>
/// <returns>Used in scenarios where one file path creates multiple JTokens, a false value indicates processing additional JTokens should be halted, proceed otherwise.</returns>
protected delegate bool JTokenProcessingDelegate(JToken token);
public override string Id { get; } = "NpmWithRoots";
public override IEnumerable<string> Categories => new[] { Enum.GetName(typeof(DetectorClass), DetectorClass.Npm) };
public override IList<string> SearchPatterns { get; } = new List<string> { "package-lock.json", "npm-shrinkwrap.json", LernaSearchPattern };
public override IEnumerable<ComponentType> SupportedComponentTypes { get; } = new[] { ComponentType.Npm };
public override int Version { get; } = 2;
public List<ProcessRequest> LernaFiles { get; set; } = new List<ProcessRequest>();
/// <inheritdoc />
protected override IList<string> SkippedFolders => new List<string> { "node_modules", "pnpm-store" };
private static string NodeModules => NpmComponentUtilities.NodeModules;
protected override Task<IObservable<ProcessRequest>> OnPrepareDetectionAsync(IObservable<ProcessRequest> processRequests, IDictionary<string, string> detectorArgs)
public NpmComponentDetectorWithRoots(IPathUtilityService pathUtilityService)
: base(pathUtilityService)
{
return Task.FromResult(this.RemoveNodeModuleNestedFiles(processRequests)
.Where(pr =>
{
if (pr.ComponentStream.Pattern.Equals(LernaSearchPattern))
{
// Lock LernaFiles so we don't add while we are enumerating in processFiles
lock (this.lernaFilesLock)
{
this.LernaFiles.Add(pr);
return false;
}
}
return true;
}));
}
protected override async Task OnFileFoundAsync(ProcessRequest processRequest, IDictionary<string, string> detectorArgs)
{
IEnumerable<string> packageJsonPattern = new List<string> { "package.json" };
var singleFileComponentRecorder = processRequest.SingleFileComponentRecorder;
var file = processRequest.ComponentStream;
public override string Id => "NpmWithRoots";
var packageJsonComponentStream = this.ComponentStreamEnumerableFactory.GetComponentStreams(new FileInfo(file.Location).Directory, packageJsonPattern, (name, directoryName) => false, false);
public override int Version => 2;
var foundUnderLerna = false;
IList<ProcessRequest> lernaFilesClone;
protected override bool IsSupportedLockfileVersion(int lockfileVersion) => lockfileVersion != 3;
// ToList LernaFiles to generate a copy we can act on without holding the lock for a long time
lock (this.lernaFilesLock)
{
lernaFilesClone = this.LernaFiles.ToList();
}
protected override JToken ResolveDependencyObject(JToken packageLockJToken) => packageLockJToken["dependencies"];
foreach (var lernaProcessRequest in lernaFilesClone)
{
var lernaFile = lernaProcessRequest.ComponentStream;
// We have extra validation on lock files not found below a lerna.json
if (this.pathUtilityService.IsFileBelowAnother(lernaFile.Location, file.Location))
{
foundUnderLerna = true;
break;
}
}
await this.SafeProcessAllPackageJTokensAsync(file, (token) =>
{
if (!foundUnderLerna && (token["name"] == null || token["version"] == null || string.IsNullOrWhiteSpace(token["name"].Value<string>()) || string.IsNullOrWhiteSpace(token["version"].Value<string>())))
{
this.Logger.LogInformation("{PackageLogJsonFile} does not contain a valid name and/or version. These are required fields for a valid package-lock.json file. It and its dependencies will not be registered.", file.Location);
return false;
}
this.ProcessIndividualPackageJTokens(singleFileComponentRecorder, token, packageJsonComponentStream, skipValidation: foundUnderLerna);
return true;
});
}
protected Task ProcessAllPackageJTokensAsync(IComponentStream componentStream, JTokenProcessingDelegate jtokenProcessor)
{
try
{
if (!componentStream.Stream.CanRead)
{
componentStream.Stream.ReadByte();
}
}
catch (Exception ex)
{
this.Logger.LogInformation(ex, "Could not read {ComponentStreamFile} file.", componentStream.Location);
return Task.CompletedTask;
}
using var file = new StreamReader(componentStream.Stream);
using var reader = new JsonTextReader(file);
var o = JToken.ReadFrom(reader);
jtokenProcessor(o);
return Task.CompletedTask;
}
protected void ProcessIndividualPackageJTokens(ISingleFileComponentRecorder singleFileComponentRecorder, JToken packageLockJToken, IEnumerable<IComponentStream> packageJsonComponentStream, bool skipValidation = false)
{
var lockfileVersion = packageLockJToken.Value<int>("lockfileVersion");
using var lockfileVersionTelemetry = new NpmLockfileVersionTelemetryRecord { LockfileVersion = lockfileVersion };
lockfileVersion = NpmComponentUtilities.UpdateLockFileVersion(lockfileVersion, this.environmentVariableService, this.Logger);
var dependencies = lockfileVersion == 3 ? packageLockJToken["packages"] : packageLockJToken["dependencies"];
var topLevelDependencies = new Queue<(JProperty, TypedComponent)>();
var dependencyLookup = dependencies.Children<JProperty>().ToDictionary(dependency => dependency.Name);
foreach (var stream in packageJsonComponentStream)
{
using var file = new StreamReader(stream.Stream);
using var reader = new JsonTextReader(file);
var packageJsonToken = JToken.ReadFrom(reader);
var enqueued = this.TryEnqueueFirstLevelDependencies(topLevelDependencies, packageJsonToken["dependencies"], dependencyLookup, lockFileVersion: lockfileVersion, skipValidation: skipValidation);
enqueued = enqueued && this.TryEnqueueFirstLevelDependencies(topLevelDependencies, packageJsonToken["devDependencies"], dependencyLookup, lockFileVersion: lockfileVersion, skipValidation: skipValidation);
if (!enqueued)
{
// This represents a mismatch between lock file and package.json, break out and do not register anything for these files
throw new InvalidOperationException(string.Format("InvalidPackageJson -- There was a mismatch between the components in the package.json and the lock file at '{0}'", singleFileComponentRecorder.ManifestFileLocation));
}
}
if (!packageJsonComponentStream.Any())
{
throw new InvalidOperationException(string.Format("InvalidPackageJson -- There must be a package.json file at '{0}' for components to be registered", singleFileComponentRecorder.ManifestFileLocation));
}
this.TraverseRequirementAndDependencyTree(topLevelDependencies, dependencyLookup, lockfileVersion, singleFileComponentRecorder);
}
private IObservable<ProcessRequest> RemoveNodeModuleNestedFiles(IObservable<ProcessRequest> componentStreams)
{
var directoryItemFacades = new List<DirectoryItemFacade>();
var directoryItemFacadesByPath = new Dictionary<string, DirectoryItemFacade>();
return Observable.Create<ProcessRequest>(s =>
{
return componentStreams.Subscribe(
processRequest =>
{
var item = processRequest.ComponentStream;
var currentDir = item.Location;
DirectoryItemFacade last = null;
do
{
currentDir = this.pathUtilityService.GetParentDirectory(currentDir);
// We've reached the top / root
if (currentDir == null)
{
// If our last directory isn't in our list of top level nodes, it should be added. This happens for the first processed item and then subsequent times we have a new root (edge cases with multiple hard drives, for example)
if (!directoryItemFacades.Contains(last))
{
directoryItemFacades.Add(last);
}
var skippedFolder = this.SkippedFolders.FirstOrDefault(folder => item.Location.Contains(folder));
// When node_modules is part of the path down to a given item, we skip the item. Otherwise, we yield the item.
if (string.IsNullOrEmpty(skippedFolder))
{
s.OnNext(processRequest);
}
else
{
this.Logger.LogDebug("Ignoring package-lock.json at {PackageLockJsonLocation}, as it is inside a {SkippedFolder} folder.", item.Location, skippedFolder);
}
break;
}
var directoryExisted = directoryItemFacadesByPath.TryGetValue(currentDir, out var current);
if (!directoryExisted)
{
directoryItemFacadesByPath[currentDir] = current = new DirectoryItemFacade
{
Name = currentDir,
Files = new List<IComponentStream>(),
Directories = new List<DirectoryItemFacade>(),
};
}
// If we came from a directory, we add it to our graph.
if (last != null)
{
current.Directories.Add(last);
}
// If we didn't come from a directory, it's because we're just getting started. Our current directory should include the file that led to it showing up in the graph.
else
{
current.Files.Add(item);
}
last = current;
}
// Go all the way up
while (currentDir != null);
},
s.OnCompleted);
});
}
private async Task SafeProcessAllPackageJTokensAsync(IComponentStream componentStream, JTokenProcessingDelegate jtokenProcessor)
{
try
{
await this.ProcessAllPackageJTokensAsync(componentStream, jtokenProcessor);
}
catch (Exception e)
{
// If something went wrong, just ignore the component
this.Logger.LogInformation(e, "Could not parse Jtokens from {ComponentLocation} file.", componentStream.Location);
}
}
private void TraverseRequirementAndDependencyTree(
IEnumerable<(JProperty Dependency, TypedComponent ParentComponent)> topLevelDependencies,
IDictionary<string, JProperty> dependencyLookup,
int lockFileVersion,
ISingleFileComponentRecorder singleFileComponentRecorder)
{
// iterate through everything for a top level dependency with a single explicitReferencedDependency value
foreach (var (currentDependency, _) in topLevelDependencies)
{
var typedComponent = NpmComponentUtilities.GetTypedComponent(currentDependency, NpmRegistryHost, this.Logger);
if (typedComponent == null)
{
continue;
}
var previouslyAddedComponents = new HashSet<string> { typedComponent.Id };
var subQueue = new Queue<(JProperty, TypedComponent)>();
NpmComponentUtilities.TraverseAndRecordComponents(currentDependency, singleFileComponentRecorder, typedComponent, explicitReferencedDependency: typedComponent);
if (lockFileVersion == 3)
{
this.TryEnqueueFirstLevelDependenciesLockfile3(subQueue, currentDependency.Value["dependencies"], dependencyLookup, singleFileComponentRecorder, parentComponent: typedComponent);
}
else
{
this.EnqueueDependencies(subQueue, currentDependency.Value["dependencies"], parentComponent: typedComponent);
this.TryEnqueueFirstLevelDependencies(subQueue, currentDependency.Value["requires"], dependencyLookup, parentComponent: typedComponent);
}
while (subQueue.Count != 0)
{
var (currentSubDependency, parentComponent) = subQueue.Dequeue();
var typedSubComponent = NpmComponentUtilities.GetTypedComponent(currentSubDependency, NpmRegistryHost, this.Logger);
// only process components that we haven't seen before that have been brought in by the same explicitReferencedDependency, resolves circular npm 'requires' loop
if (typedSubComponent == null || previouslyAddedComponents.Contains(typedSubComponent.Id))
{
continue;
}
previouslyAddedComponents.Add(typedSubComponent.Id);
NpmComponentUtilities.TraverseAndRecordComponents(currentSubDependency, singleFileComponentRecorder, typedSubComponent, explicitReferencedDependency: typedComponent, parentComponent.Id);
if (lockFileVersion == 3)
{
this.TryEnqueueFirstLevelDependenciesLockfile3(subQueue, currentSubDependency.Value["dependencies"], dependencyLookup, singleFileComponentRecorder, parentComponent: typedSubComponent);
}
else
{
this.EnqueueDependencies(subQueue, currentSubDependency.Value["dependencies"], parentComponent: typedSubComponent);
this.TryEnqueueFirstLevelDependencies(subQueue, currentSubDependency.Value["requires"], dependencyLookup, parentComponent: typedSubComponent);
}
}
}
}
private void EnqueueDependencies(Queue<(JProperty Dependency, TypedComponent ParentComponent)> queue, JToken dependencies, TypedComponent parentComponent)
{
if (dependencies != null)
{
foreach (var dependency in dependencies.Cast<JProperty>())
{
if (dependency != null)
{
queue.Enqueue((dependency, parentComponent));
}
}
}
}
private bool TryEnqueueFirstLevelDependencies(
protected override bool TryEnqueueFirstLevelDependencies(
Queue<(JProperty DependencyProperty, TypedComponent ParentComponent)> queue,
JToken dependencies,
IDictionary<string, JProperty> dependencyLookup,
int lockFileVersion = 2,
Queue<TypedComponent> parentComponentQueue = null,
TypedComponent parentComponent = null,
bool skipValidation = false)
{
@ -370,7 +56,7 @@ public class NpmComponentDetectorWithRoots : FileComponentDetector
continue;
}
var inLock = dependencyLookup.TryGetValue(lockFileVersion == 3 ? $"{NodeModules}/{dependency.Name}" : dependency.Name, out var dependencyProperty);
var inLock = dependencyLookup.TryGetValue(dependency.Name, out var dependencyProperty);
if (inLock)
{
queue.Enqueue((dependencyProperty, parentComponent));
@ -387,78 +73,30 @@ public class NpmComponentDetectorWithRoots : FileComponentDetector
return isValid;
}
private bool TryEnqueueFirstLevelDependenciesLockfile3(
Queue<(JProperty DependencyProperty, TypedComponent ParentComponent)> queue,
JToken dependencies,
protected override void EnqueueAllDependencies(
IDictionary<string, JProperty> dependencyLookup,
ISingleFileComponentRecorder componentRecorder,
TypedComponent parentComponent)
ISingleFileComponentRecorder singleFileComponentRecorder,
Queue<(JProperty CurrentSubDependency, TypedComponent ParentComponent)> subQueue,
JProperty currentDependency,
TypedComponent typedComponent)
{
this.EnqueueDependencies(subQueue, currentDependency.Value["dependencies"], parentComponent: typedComponent);
this.TryEnqueueFirstLevelDependencies(subQueue, currentDependency.Value["requires"], dependencyLookup, parentComponent: typedComponent);
}
private void EnqueueDependencies(Queue<(JProperty Dependency, TypedComponent ParentComponent)> queue, JToken dependencies, TypedComponent parentComponent)
{
if (dependencies == null)
{
return true;
return;
}
var isValid = true;
foreach (var dependency in dependencies.Cast<JProperty>())
{
if (dependency?.Name == null)
if (dependency != null)
{
continue;
}
// First, check if there is an entry in the lockfile for this dependency nested in its ancestors
var ancestors = componentRecorder.DependencyGraph.GetAncestors(parentComponent.Id);
ancestors.Add(parentComponent.Id);
// remove version information
ancestors = ancestors.Select(x => x.Split(' ')[0]).ToList();
var possibleDepPaths = ancestors
.Select((t, i) => ancestors.TakeLast(ancestors.Count - i)); // depth-first search
var inLock = false;
JProperty dependencyProperty;
foreach (var possibleDepPath in possibleDepPaths)
{
var ancestorNodeModulesPath = string.Format(
"{0}/{1}/{0}/{2}",
NodeModules,
string.Join($"/{NodeModules}/", possibleDepPath),
dependency.Name);
// Does this exist?
inLock = dependencyLookup.TryGetValue(ancestorNodeModulesPath, out dependencyProperty);
if (!inLock)
{
continue;
}
this.Logger.LogDebug("Found nested dependency {Dependency} in {AncestorNodeModulesPath}", dependency.Name, ancestorNodeModulesPath);
queue.Enqueue((dependencyProperty, parentComponent));
break;
}
if (inLock)
{
continue;
}
// If not, check if there is an entry in the lockfile for this dependency at the top level
inLock = dependencyLookup.TryGetValue($"{NodeModules}/{dependency.Name}", out dependencyProperty);
if (inLock)
{
queue.Enqueue((dependencyProperty, parentComponent));
}
else
{
this.Logger.LogWarning("Could not find dependency {Dependency} in lockfile", dependency.Name);
isValid = false;
queue.Enqueue((dependency, parentComponent));
}
}
return isValid;
}
}

Просмотреть файл

@ -190,31 +190,4 @@ public static class NpmComponentUtilities
|| name.StartsWith('_')
|| UnsafeCharactersRegex.IsMatch(name));
}
/// <summary>
/// Updates the lockfile version based on the a feature gate environment variable. If the lock file version is 3,
/// and the environment variable <see cref="LockFile3EnvFlag"/> is not set, the lock file version is downgraded to 2.
/// </summary>
/// <param name="lockfileVersion">The lockfileVersion read from the package-lock.json.</param>
/// <param name="envService">The environment variable service.</param>
/// <param name="logger">The logger.</param>
/// <returns>The lockfileVersion to treat the package lock as.</returns>
public static int UpdateLockFileVersion(int lockfileVersion, IEnvironmentVariableService envService, ILogger logger)
{
if (lockfileVersion != 3)
{
return lockfileVersion;
}
var envVarSet =
!string.IsNullOrEmpty(envService.GetEnvironmentVariable(LockFile3EnvFlag));
if (!envVarSet)
{
return 2;
}
logger.LogInformation("Enabling experimental NPM lockfile v3 support");
return lockfileVersion; // Lockfile v3 is enabled
}
}

Просмотреть файл

@ -0,0 +1,160 @@
namespace Microsoft.ComponentDetection.Detectors.Npm;
using System.Collections.Generic;
using System.Linq;
using Microsoft.ComponentDetection.Contracts;
using Microsoft.ComponentDetection.Contracts.TypedComponent;
using Microsoft.Extensions.Logging;
using Newtonsoft.Json.Linq;
public class NpmLockfile3Detector : NpmLockfileDetectorBase, IExperimentalDetector
{
private static readonly string NodeModules = NpmComponentUtilities.NodeModules;
public NpmLockfile3Detector(
IComponentStreamEnumerableFactory componentStreamEnumerableFactory,
IObservableDirectoryWalkerFactory walkerFactory,
IPathUtilityService pathUtilityService,
ILogger<NpmLockfile3Detector> logger)
: base(
componentStreamEnumerableFactory,
walkerFactory,
pathUtilityService,
logger)
{
}
public NpmLockfile3Detector(IPathUtilityService pathUtilityService)
: base(pathUtilityService)
{
}
public override string Id => "NpmLockfile3";
public override int Version => 1;
protected override bool IsSupportedLockfileVersion(int lockfileVersion) => lockfileVersion == 3;
protected override JToken ResolveDependencyObject(JToken packageLockJToken) => packageLockJToken["packages"];
protected override bool TryEnqueueFirstLevelDependencies(
Queue<(JProperty DependencyProperty, TypedComponent ParentComponent)> queue,
JToken dependencies,
IDictionary<string, JProperty> dependencyLookup,
TypedComponent parentComponent = null,
bool skipValidation = false)
{
if (dependencies == null)
{
return true;
}
var isValid = true;
foreach (var dependency in dependencies.Cast<JProperty>())
{
if (dependency?.Name == null)
{
continue;
}
var inLock = dependencyLookup.TryGetValue($"{NodeModules}/{dependency.Name}", out var dependencyProperty);
if (inLock)
{
queue.Enqueue((dependencyProperty, parentComponent));
}
else if (skipValidation)
{
}
else
{
isValid = false;
}
}
return isValid;
}
protected override void EnqueueAllDependencies(
IDictionary<string, JProperty> dependencyLookup,
ISingleFileComponentRecorder singleFileComponentRecorder,
Queue<(JProperty CurrentSubDependency, TypedComponent ParentComponent)> subQueue,
JProperty currentDependency,
TypedComponent typedComponent) =>
this.TryEnqueueFirstLevelDependenciesLockfile3(
subQueue,
currentDependency.Value["dependencies"],
dependencyLookup,
singleFileComponentRecorder,
parentComponent: typedComponent);
private void TryEnqueueFirstLevelDependenciesLockfile3(
Queue<(JProperty DependencyProperty, TypedComponent ParentComponent)> queue,
JToken dependencies,
IDictionary<string, JProperty> dependencyLookup,
ISingleFileComponentRecorder componentRecorder,
TypedComponent parentComponent)
{
if (dependencies == null)
{
return;
}
foreach (var dependency in dependencies.Cast<JProperty>())
{
if (dependency?.Name == null)
{
continue;
}
// First, check if there is an entry in the lockfile for this dependency nested in its ancestors
var ancestors = componentRecorder.DependencyGraph.GetAncestors(parentComponent.Id);
ancestors.Add(parentComponent.Id);
// remove version information
ancestors = ancestors.Select(x => x.Split(' ')[0]).ToList();
var possibleDepPaths = ancestors
.Select((t, i) => ancestors.TakeLast(ancestors.Count - i)); // depth-first search
var inLock = false;
JProperty dependencyProperty;
foreach (var possibleDepPath in possibleDepPaths)
{
var ancestorNodeModulesPath = string.Format(
"{0}/{1}/{0}/{2}",
NodeModules,
string.Join($"/{NodeModules}/", possibleDepPath),
dependency.Name);
// Does this exist?
inLock = dependencyLookup.TryGetValue(ancestorNodeModulesPath, out dependencyProperty);
if (!inLock)
{
continue;
}
this.Logger.LogDebug("Found nested dependency {Dependency} in {AncestorNodeModulesPath}", dependency.Name, ancestorNodeModulesPath);
queue.Enqueue((dependencyProperty, parentComponent));
break;
}
if (inLock)
{
continue;
}
// If not, check if there is an entry in the lockfile for this dependency at the top level
inLock = dependencyLookup.TryGetValue($"{NodeModules}/{dependency.Name}", out dependencyProperty);
if (inLock)
{
queue.Enqueue((dependencyProperty, parentComponent));
}
else
{
this.Logger.LogWarning("Could not find dependency {Dependency} in lockfile", dependency.Name);
}
}
}
}

Просмотреть файл

@ -0,0 +1,323 @@
namespace Microsoft.ComponentDetection.Detectors.Npm;
using System;
using System.Collections.Generic;
using System.IO;
using System.Linq;
using System.Reactive.Linq;
using System.Threading.Tasks;
using Microsoft.ComponentDetection.Common;
using Microsoft.ComponentDetection.Common.Telemetry.Records;
using Microsoft.ComponentDetection.Contracts;
using Microsoft.ComponentDetection.Contracts.Internal;
using Microsoft.ComponentDetection.Contracts.TypedComponent;
using Microsoft.Extensions.Logging;
using Newtonsoft.Json;
using Newtonsoft.Json.Linq;
public abstract class NpmLockfileDetectorBase : FileComponentDetector
{
private const string NpmRegistryHost = "registry.npmjs.org";
private const string LernaSearchPattern = "lerna.json";
private readonly object lernaFilesLock = new object();
/// <summary>
/// Gets or sets the logger for writing basic logging message to both console and file.
/// </summary>
private readonly IPathUtilityService pathUtilityService;
protected NpmLockfileDetectorBase(
IComponentStreamEnumerableFactory componentStreamEnumerableFactory,
IObservableDirectoryWalkerFactory walkerFactory,
IPathUtilityService pathUtilityService,
ILogger logger)
{
this.ComponentStreamEnumerableFactory = componentStreamEnumerableFactory;
this.Scanner = walkerFactory;
this.pathUtilityService = pathUtilityService;
this.Logger = logger;
}
protected NpmLockfileDetectorBase(IPathUtilityService pathUtilityService) => this.pathUtilityService = pathUtilityService;
/// <summary>Common delegate for Package.json JToken processing.</summary>
/// <param name="token">A JToken, usually corresponding to a package.json file.</param>
/// <returns>Used in scenarios where one file path creates multiple JTokens, a false value indicates processing additional JTokens should be halted, proceed otherwise.</returns>
protected delegate bool JTokenProcessingDelegate(JToken token);
public override IEnumerable<string> Categories => new[] { Enum.GetName(typeof(DetectorClass), DetectorClass.Npm) };
public override IList<string> SearchPatterns { get; } = new List<string> { "package-lock.json", "npm-shrinkwrap.json", LernaSearchPattern };
public override IEnumerable<ComponentType> SupportedComponentTypes { get; } = new[] { ComponentType.Npm };
private List<ProcessRequest> LernaFiles { get; } = new();
/// <inheritdoc />
protected override IList<string> SkippedFolders => new List<string> { "node_modules", "pnpm-store" };
protected abstract bool IsSupportedLockfileVersion(int lockfileVersion);
protected abstract JToken ResolveDependencyObject(JToken packageLockJToken);
protected abstract void EnqueueAllDependencies(
IDictionary<string, JProperty> dependencyLookup,
ISingleFileComponentRecorder singleFileComponentRecorder,
Queue<(JProperty CurrentSubDependency, TypedComponent ParentComponent)> subQueue,
JProperty currentDependency,
TypedComponent typedComponent);
protected abstract bool TryEnqueueFirstLevelDependencies(
Queue<(JProperty DependencyProperty, TypedComponent ParentComponent)> queue,
JToken dependencies,
IDictionary<string, JProperty> dependencyLookup,
TypedComponent parentComponent = null,
bool skipValidation = false);
protected override Task<IObservable<ProcessRequest>> OnPrepareDetectionAsync(IObservable<ProcessRequest> processRequests, IDictionary<string, string> detectorArgs) =>
Task.FromResult(this.RemoveNodeModuleNestedFiles(processRequests)
.Where(pr =>
{
if (!pr.ComponentStream.Pattern.Equals(LernaSearchPattern))
{
return true;
}
// Lock LernaFiles so we don't add while we are enumerating in processFiles
lock (this.lernaFilesLock)
{
this.LernaFiles.Add(pr);
return false;
}
}));
protected override async Task OnFileFoundAsync(ProcessRequest processRequest, IDictionary<string, string> detectorArgs)
{
IEnumerable<string> packageJsonPattern = new List<string> { "package.json" };
var singleFileComponentRecorder = processRequest.SingleFileComponentRecorder;
var file = processRequest.ComponentStream;
var packageJsonComponentStream = this.ComponentStreamEnumerableFactory.GetComponentStreams(new FileInfo(file.Location).Directory, packageJsonPattern, (name, directoryName) => false, false);
IList<ProcessRequest> lernaFilesClone;
// ToList LernaFiles to generate a copy we can act on without holding the lock for a long time
lock (this.lernaFilesLock)
{
lernaFilesClone = this.LernaFiles.ToList();
}
var foundUnderLerna = lernaFilesClone.Select(lernaProcessRequest => lernaProcessRequest.ComponentStream)
.Any(lernaFile => this.pathUtilityService.IsFileBelowAnother(
lernaFile.Location,
file.Location));
await this.SafeProcessAllPackageJTokensAsync(file, (token) =>
{
if (!foundUnderLerna &&
(token["name"] == null ||
token["version"] == null ||
string.IsNullOrWhiteSpace(token["name"].Value<string>()) ||
string.IsNullOrWhiteSpace(token["version"].Value<string>())))
{
this.Logger.LogInformation("{PackageLogJsonFile} does not contain a valid name and/or version. These are required fields for a valid package-lock.json file. It and its dependencies will not be registered.", file.Location);
return false;
}
this.ProcessIndividualPackageJTokens(singleFileComponentRecorder, token, packageJsonComponentStream, skipValidation: foundUnderLerna);
return true;
});
}
protected Task ProcessAllPackageJTokensAsync(IComponentStream componentStream, JTokenProcessingDelegate jtokenProcessor)
{
try
{
if (!componentStream.Stream.CanRead)
{
componentStream.Stream.ReadByte();
}
}
catch (Exception ex)
{
this.Logger.LogInformation(ex, "Could not read {ComponentStreamFile} file.", componentStream.Location);
return Task.CompletedTask;
}
using var file = new StreamReader(componentStream.Stream);
using var reader = new JsonTextReader(file);
var o = JToken.ReadFrom(reader);
jtokenProcessor(o);
return Task.CompletedTask;
}
private void ProcessIndividualPackageJTokens(ISingleFileComponentRecorder singleFileComponentRecorder, JToken packageLockJToken, IEnumerable<IComponentStream> packageJsonComponentStream, bool skipValidation = false)
{
var lockfileVersion = packageLockJToken.Value<int>("lockfileVersion");
using var lockfileVersionTelemetry = new NpmLockfileVersionTelemetryRecord { LockfileVersion = lockfileVersion };
if (!this.IsSupportedLockfileVersion(lockfileVersion))
{
return;
}
var dependencies = this.ResolveDependencyObject(packageLockJToken);
var topLevelDependencies = new Queue<(JProperty, TypedComponent)>();
var dependencyLookup = dependencies.Children<JProperty>().ToDictionary(dependency => dependency.Name);
foreach (var stream in packageJsonComponentStream)
{
using var file = new StreamReader(stream.Stream);
using var reader = new JsonTextReader(file);
var packageJsonToken = JToken.ReadFrom(reader);
var enqueued = this.TryEnqueueFirstLevelDependencies(topLevelDependencies, packageJsonToken["dependencies"], dependencyLookup, skipValidation: skipValidation);
enqueued = enqueued && this.TryEnqueueFirstLevelDependencies(topLevelDependencies, packageJsonToken["devDependencies"], dependencyLookup, skipValidation: skipValidation);
if (!enqueued)
{
// This represents a mismatch between lock file and package.json, break out and do not register anything for these files
throw new InvalidOperationException(string.Format("InvalidPackageJson -- There was a mismatch between the components in the package.json and the lock file at '{0}'", singleFileComponentRecorder.ManifestFileLocation));
}
}
if (!packageJsonComponentStream.Any())
{
throw new InvalidOperationException(string.Format("InvalidPackageJson -- There must be a package.json file at '{0}' for components to be registered", singleFileComponentRecorder.ManifestFileLocation));
}
this.TraverseRequirementAndDependencyTree(topLevelDependencies, dependencyLookup, singleFileComponentRecorder);
}
private IObservable<ProcessRequest> RemoveNodeModuleNestedFiles(IObservable<ProcessRequest> componentStreams)
{
var directoryItemFacades = new List<DirectoryItemFacade>();
var directoryItemFacadesByPath = new Dictionary<string, DirectoryItemFacade>();
return Observable.Create<ProcessRequest>(s =>
{
return componentStreams.Subscribe(
processRequest =>
{
var item = processRequest.ComponentStream;
var currentDir = item.Location;
DirectoryItemFacade last = null;
do
{
currentDir = this.pathUtilityService.GetParentDirectory(currentDir);
// We've reached the top / root
if (currentDir == null)
{
// If our last directory isn't in our list of top level nodes, it should be added. This happens for the first processed item and then subsequent times we have a new root (edge cases with multiple hard drives, for example)
if (!directoryItemFacades.Contains(last))
{
directoryItemFacades.Add(last);
}
var skippedFolder = this.SkippedFolders.FirstOrDefault(folder => item.Location.Contains(folder));
// When node_modules is part of the path down to a given item, we skip the item. Otherwise, we yield the item.
if (string.IsNullOrEmpty(skippedFolder))
{
s.OnNext(processRequest);
}
else
{
this.Logger.LogDebug("Ignoring package-lock.json at {PackageLockJsonLocation}, as it is inside a {SkippedFolder} folder.", item.Location, skippedFolder);
}
break;
}
var directoryExisted = directoryItemFacadesByPath.TryGetValue(currentDir, out var current);
if (!directoryExisted)
{
directoryItemFacadesByPath[currentDir] = current = new DirectoryItemFacade
{
Name = currentDir,
Files = new List<IComponentStream>(),
Directories = new List<DirectoryItemFacade>(),
};
}
// If we came from a directory, we add it to our graph.
if (last != null)
{
current.Directories.Add(last);
}
// If we didn't come from a directory, it's because we're just getting started. Our current directory should include the file that led to it showing up in the graph.
else
{
current.Files.Add(item);
}
last = current;
}
// Go all the way up
while (currentDir != null);
},
s.OnCompleted);
});
}
private async Task SafeProcessAllPackageJTokensAsync(IComponentStream componentStream, JTokenProcessingDelegate jtokenProcessor)
{
try
{
await this.ProcessAllPackageJTokensAsync(componentStream, jtokenProcessor);
}
catch (Exception e)
{
// If something went wrong, just ignore the component
this.Logger.LogInformation(e, "Could not parse Jtokens from {ComponentLocation} file.", componentStream.Location);
}
}
private void TraverseRequirementAndDependencyTree(
IEnumerable<(JProperty Dependency, TypedComponent ParentComponent)> topLevelDependencies,
IDictionary<string, JProperty> dependencyLookup,
ISingleFileComponentRecorder singleFileComponentRecorder)
{
// iterate through everything for a top level dependency with a single explicitReferencedDependency value
foreach (var (currentDependency, _) in topLevelDependencies)
{
var typedComponent = NpmComponentUtilities.GetTypedComponent(currentDependency, NpmRegistryHost, this.Logger);
if (typedComponent == null)
{
continue;
}
var previouslyAddedComponents = new HashSet<string> { typedComponent.Id };
var subQueue = new Queue<(JProperty, TypedComponent)>();
NpmComponentUtilities.TraverseAndRecordComponents(currentDependency, singleFileComponentRecorder, typedComponent, explicitReferencedDependency: typedComponent);
this.EnqueueAllDependencies(dependencyLookup, singleFileComponentRecorder, subQueue, currentDependency, typedComponent);
while (subQueue.Count != 0)
{
var (currentSubDependency, parentComponent) = subQueue.Dequeue();
var typedSubComponent = NpmComponentUtilities.GetTypedComponent(currentSubDependency, NpmRegistryHost, this.Logger);
// only process components that we haven't seen before that have been brought in by the same explicitReferencedDependency, resolves circular npm 'requires' loop
if (typedSubComponent == null || previouslyAddedComponents.Contains(typedSubComponent.Id))
{
continue;
}
previouslyAddedComponents.Add(typedSubComponent.Id);
NpmComponentUtilities.TraverseAndRecordComponents(currentSubDependency, singleFileComponentRecorder, typedSubComponent, explicitReferencedDependency: typedComponent, parentComponent.Id);
this.EnqueueAllDependencies(dependencyLookup, singleFileComponentRecorder, subQueue, currentSubDependency, typedSubComponent);
}
}
}
}

Просмотреть файл

@ -93,6 +93,7 @@ public static class ServiceCollectionExtensions
// npm
services.AddSingleton<IComponentDetector, NpmComponentDetector>();
services.AddSingleton<IComponentDetector, NpmComponentDetectorWithRoots>();
services.AddSingleton<IComponentDetector, NpmLockfile3Detector>();
// NuGet
services.AddSingleton<IComponentDetector, NuGetComponentDetector>();

Просмотреть файл

@ -25,14 +25,11 @@ public class NpmDetectorWithRootsTests : BaseDetectorTest<NpmComponentDetectorWi
private readonly List<string> packageJsonSearchPattern = new() { "package.json" };
private readonly List<string> packageLockJsonSearchPatterns = new() { "package-lock.json", "npm-shrinkwrap.json", "lerna.json" };
private readonly Mock<IPathUtilityService> mockPathUtilityService;
private readonly Mock<IEnvironmentVariableService> mockEnvService;
public NpmDetectorWithRootsTests()
{
this.mockPathUtilityService = new Mock<IPathUtilityService>();
this.mockEnvService = new Mock<IEnvironmentVariableService>();
this.DetectorTestUtility.AddServiceMock(this.mockPathUtilityService);
this.DetectorTestUtility.AddServiceMock(this.mockEnvService);
}
[TestMethod]
@ -61,97 +58,6 @@ public class NpmDetectorWithRootsTests : BaseDetectorTest<NpmComponentDetectorWi
}
}
[TestMethod]
public async Task TestNpmDetector_PackageLockVersion3ReturnsValidAsync()
{
this.mockEnvService
.Setup(x =>
x.GetEnvironmentVariable(NpmComponentUtilities.LockFile3EnvFlag))
.Returns("true");
var componentName0 = Guid.NewGuid().ToString("N");
var version0 = NewRandomVersion();
var (packageLockName, packageLockContents, packageLockPath) = NpmTestUtilities.GetWellFormedPackageLock3(this.packageLockJsonFileName, componentName0, version0);
var (packageJsonName, packageJsonContents, packageJsonPath) = NpmTestUtilities.GetPackageJsonOneRoot(componentName0, version0);
var (scanResult, componentRecorder) = await this.DetectorTestUtility
.WithFile(packageLockName, packageLockContents, this.packageLockJsonSearchPatterns, fileLocation: packageLockPath)
.WithFile(packageJsonName, packageJsonContents, this.packageJsonSearchPattern, fileLocation: packageJsonPath)
.ExecuteDetectorAsync();
Assert.AreEqual(ProcessingResultCode.Success, scanResult.ResultCode);
var detectedComponents = componentRecorder.GetDetectedComponents();
Assert.AreEqual(4, detectedComponents.Count());
foreach (var component in detectedComponents)
{
componentRecorder.AssertAllExplicitlyReferencedComponents<NpmComponent>(
component.Component.Id,
parentComponent0 => parentComponent0.Name == componentName0 && parentComponent0.Version == version0);
Assert.IsFalse(string.IsNullOrWhiteSpace(((NpmComponent)component.Component).Hash));
}
}
[TestMethod]
public async Task TestNpmDetector_PackageLockVersion3NestedReturnsValidAsync()
{
this.mockEnvService
.Setup(x =>
x.GetEnvironmentVariable(NpmComponentUtilities.LockFile3EnvFlag))
.Returns("true");
var componentName0 = Guid.NewGuid().ToString("N");
var version0 = NewRandomVersion();
var componentName1 = Guid.NewGuid().ToString("N");
var version1 = NewRandomVersion();
var componentName2 = Guid.NewGuid().ToString("N");
var (packageLockName, packageLockContents, packageLockPath) = NpmTestUtilities.GetWellFormedNestedPackageLock3(this.packageLockJsonFileName, componentName0, version0, componentName1, version1, componentName2);
var packagejson = @"{{
""name"": ""test"",
""version"": ""0.0.0"",
""dependencies"": {{
""{0}"": ""{1}"",
""{2}"": ""{3}""
}}
}}";
var packageJsonTemplate = string.Format(packagejson, componentName0, version0, componentName1, version1);
var (scanResult, componentRecorder) = await this.DetectorTestUtility
.WithFile(packageLockName, packageLockContents, this.packageLockJsonSearchPatterns, fileLocation: packageLockPath)
.WithFile(this.packageJsonFileName, packageJsonTemplate, this.packageJsonSearchPattern)
.ExecuteDetectorAsync();
Assert.AreEqual(ProcessingResultCode.Success, scanResult.ResultCode);
var detectedComponents = componentRecorder.GetDetectedComponents().ToList();
Assert.AreEqual(4, detectedComponents.Count);
var component0 = detectedComponents.First(x => x.Component.Id.Contains(componentName0));
componentRecorder.AssertAllExplicitlyReferencedComponents<NpmComponent>(
component0.Component.Id,
parentComponent0 => parentComponent0.Name == componentName0);
var component1 = detectedComponents.First(x => x.Component.Id.Contains(componentName1));
componentRecorder.AssertAllExplicitlyReferencedComponents<NpmComponent>(
component1.Component.Id,
parentComponent0 => parentComponent0.Name == componentName1);
var duplicate = detectedComponents.Where(x => x.Component.Id.Contains(componentName2)).ToList();
duplicate.Should().HaveCount(2);
foreach (var component in detectedComponents)
{
// check that either component0 or component1 is our parent
componentRecorder.IsDependencyOfExplicitlyReferencedComponents<NpmComponent>(
component.Component.Id,
parentComponent0 => parentComponent0.Name == componentName0 || parentComponent0.Name == componentName1);
Assert.IsFalse(string.IsNullOrWhiteSpace(((NpmComponent)component.Component).Hash));
}
}
[TestMethod]
public async Task TestNpmDetector_MismatchedFilesReturnsEmptyAsync()
{
@ -674,76 +580,4 @@ public class NpmDetectorWithRootsTests : BaseDetectorTest<NpmComponentDetectorWi
dependencyGraph.GetDependenciesForComponent(componentBId).Should().Contain(componentCId);
dependencyGraph.GetDependenciesForComponent(componentCId).Should().HaveCount(0);
}
[TestMethod]
public async Task TestNpmDetector_NestedNodeModulesV3Async()
{
this.mockEnvService
.Setup(x =>
x.GetEnvironmentVariable(NpmComponentUtilities.LockFile3EnvFlag))
.Returns("true");
var componentA = (Name: "componentA", Version: "1.0.0");
var componentB = (Name: "componentB", Version: "1.0.0");
var packageLockJson = @"{{
""name"": ""test"",
""version"": ""0.0.0"",
""lockfileVersion"": 3,
""requires"": true,
""packages"": {{
"""": {{
""name"": ""test"",
""version"": ""0.0.0"",
""dependencies"": {{
""{0}"": ""{1}""
}}
}},
""node_modules/{0}"": {{
""version"": ""{1}"",
""resolved"": ""https://mseng.pkgs.visualstudio.com/_packaging/VsoMicrosoftExternals/npm/registry/"",
""integrity"": ""sha1-EBPRBRBH3TIP4k5JTVxm7K9hR9k="",
""dependencies"": {{
""{2}"": ""{3}""
}}
}},
""node_modules/{0}/node_modules/{2}"": {{
""version"": ""{3}"",
""resolved"": ""https://mseng.pkgs.visualstudio.com/_packaging/VsoMicrosoftExternals/npm/registry/"",
""integrity"": ""sha1-PRT306DRK/NZUaVL07iuqH7nWPg=""
}}
}}
}}";
var packageLockTemplate = string.Format(packageLockJson, componentA.Name, componentA.Version, componentB.Name, componentB.Version);
var packagejson = @"{{
""name"": ""test"",
""version"": ""0.0.0"",
""dependencies"": {{
""{0}"": ""{1}"",
}}
}}";
var packageJsonTemplate = string.Format(packagejson, componentA.Name, componentA.Version);
var (scanResult, componentRecorder) = await this.DetectorTestUtility
.WithFile(this.packageLockJsonFileName, packageLockTemplate, this.packageLockJsonSearchPatterns)
.WithFile(this.packageJsonFileName, packageJsonTemplate, this.packageJsonSearchPattern)
.ExecuteDetectorAsync();
scanResult.ResultCode.Should().Be(ProcessingResultCode.Success);
var detectedComponents = componentRecorder.GetDetectedComponents();
detectedComponents.Should().HaveCount(2);
var componentAId = detectedComponents.First(c => ((NpmComponent)c.Component).Name.Equals(componentA.Name)).Component.Id;
var componentBId = detectedComponents.First(c => ((NpmComponent)c.Component).Name.Equals(componentB.Name)).Component.Id;
var dependencyGraph = componentRecorder.GetDependencyGraphsByLocation().Values.First();
dependencyGraph.GetDependenciesForComponent(componentAId).Should().HaveCount(1);
dependencyGraph.GetDependenciesForComponent(componentAId).Should().Contain(componentBId);
dependencyGraph.GetDependenciesForComponent(componentBId).Should().HaveCount(0);
}
}

Просмотреть файл

@ -0,0 +1,181 @@
namespace Microsoft.ComponentDetection.Detectors.Tests;
using System;
using System.Collections.Generic;
using System.Linq;
using System.Threading.Tasks;
using FluentAssertions;
using Microsoft.ComponentDetection.Contracts;
using Microsoft.ComponentDetection.Contracts.TypedComponent;
using Microsoft.ComponentDetection.Detectors.Npm;
using Microsoft.ComponentDetection.Detectors.Tests.Utilities;
using Microsoft.ComponentDetection.TestsUtilities;
using Microsoft.VisualStudio.TestTools.UnitTesting;
using Moq;
using static Microsoft.ComponentDetection.Detectors.Tests.Utilities.TestUtilityExtensions;
[TestClass]
[TestCategory("Governance/All")]
[TestCategory("Governance/ComponentDetection")]
public class NpmLockfile3DetectorTests : BaseDetectorTest<NpmLockfile3Detector>
{
private readonly string packageLockJsonFileName = "package-lock.json";
private readonly string packageJsonFileName = "package.json";
private readonly List<string> packageJsonSearchPattern = new() { "package.json" };
private readonly List<string> packageLockJsonSearchPatterns = new() { "package-lock.json", "npm-shrinkwrap.json", "lerna.json" };
private readonly Mock<IPathUtilityService> mockPathUtilityService;
public NpmLockfile3DetectorTests()
{
this.mockPathUtilityService = new Mock<IPathUtilityService>();
this.DetectorTestUtility.AddServiceMock(this.mockPathUtilityService);
}
[TestMethod]
public async Task TestNpmDetector_PackageLockVersion3ReturnsValidAsync()
{
var componentName0 = Guid.NewGuid().ToString("N");
var version0 = NewRandomVersion();
var (packageLockName, packageLockContents, packageLockPath) = NpmTestUtilities.GetWellFormedPackageLock3(this.packageLockJsonFileName, componentName0, version0);
var (packageJsonName, packageJsonContents, packageJsonPath) = NpmTestUtilities.GetPackageJsonOneRoot(componentName0, version0);
var (scanResult, componentRecorder) = await this.DetectorTestUtility
.WithFile(packageLockName, packageLockContents, this.packageLockJsonSearchPatterns, fileLocation: packageLockPath)
.WithFile(packageJsonName, packageJsonContents, this.packageJsonSearchPattern, fileLocation: packageJsonPath)
.ExecuteDetectorAsync();
Assert.AreEqual(ProcessingResultCode.Success, scanResult.ResultCode);
var detectedComponents = componentRecorder.GetDetectedComponents();
Assert.AreEqual(4, detectedComponents.Count());
foreach (var component in detectedComponents)
{
componentRecorder.AssertAllExplicitlyReferencedComponents<NpmComponent>(
component.Component.Id,
parentComponent0 => parentComponent0.Name == componentName0 && parentComponent0.Version == version0);
Assert.IsFalse(string.IsNullOrWhiteSpace(((NpmComponent)component.Component).Hash));
}
}
[TestMethod]
public async Task TestNpmDetector_PackageLockVersion3NestedReturnsValidAsync()
{
var componentName0 = Guid.NewGuid().ToString("N");
var version0 = NewRandomVersion();
var componentName1 = Guid.NewGuid().ToString("N");
var version1 = NewRandomVersion();
var componentName2 = Guid.NewGuid().ToString("N");
var (packageLockName, packageLockContents, packageLockPath) = NpmTestUtilities.GetWellFormedNestedPackageLock3(this.packageLockJsonFileName, componentName0, version0, componentName1, version1, componentName2);
var packagejson = @"{{
""name"": ""test"",
""version"": ""0.0.0"",
""dependencies"": {{
""{0}"": ""{1}"",
""{2}"": ""{3}""
}}
}}";
var packageJsonTemplate = string.Format(packagejson, componentName0, version0, componentName1, version1);
var (scanResult, componentRecorder) = await this.DetectorTestUtility
.WithFile(packageLockName, packageLockContents, this.packageLockJsonSearchPatterns, fileLocation: packageLockPath)
.WithFile(this.packageJsonFileName, packageJsonTemplate, this.packageJsonSearchPattern)
.ExecuteDetectorAsync();
Assert.AreEqual(ProcessingResultCode.Success, scanResult.ResultCode);
var detectedComponents = componentRecorder.GetDetectedComponents().ToList();
Assert.AreEqual(4, detectedComponents.Count);
var component0 = detectedComponents.First(x => x.Component.Id.Contains(componentName0));
componentRecorder.AssertAllExplicitlyReferencedComponents<NpmComponent>(
component0.Component.Id,
parentComponent0 => parentComponent0.Name == componentName0);
var component1 = detectedComponents.First(x => x.Component.Id.Contains(componentName1));
componentRecorder.AssertAllExplicitlyReferencedComponents<NpmComponent>(
component1.Component.Id,
parentComponent0 => parentComponent0.Name == componentName1);
var duplicate = detectedComponents.Where(x => x.Component.Id.Contains(componentName2)).ToList();
duplicate.Should().HaveCount(2);
foreach (var component in detectedComponents)
{
// check that either component0 or component1 is our parent
componentRecorder.IsDependencyOfExplicitlyReferencedComponents<NpmComponent>(
component.Component.Id,
parentComponent0 => parentComponent0.Name == componentName0 || parentComponent0.Name == componentName1);
Assert.IsFalse(string.IsNullOrWhiteSpace(((NpmComponent)component.Component).Hash));
}
}
[TestMethod]
public async Task TestNpmDetector_NestedNodeModulesV3Async()
{
var componentA = (Name: "componentA", Version: "1.0.0");
var componentB = (Name: "componentB", Version: "1.0.0");
var packageLockJson = @"{{
""name"": ""test"",
""version"": ""0.0.0"",
""lockfileVersion"": 3,
""requires"": true,
""packages"": {{
"""": {{
""name"": ""test"",
""version"": ""0.0.0"",
""dependencies"": {{
""{0}"": ""{1}""
}}
}},
""node_modules/{0}"": {{
""version"": ""{1}"",
""resolved"": ""https://mseng.pkgs.visualstudio.com/_packaging/VsoMicrosoftExternals/npm/registry/"",
""integrity"": ""sha1-EBPRBRBH3TIP4k5JTVxm7K9hR9k="",
""dependencies"": {{
""{2}"": ""{3}""
}}
}},
""node_modules/{0}/node_modules/{2}"": {{
""version"": ""{3}"",
""resolved"": ""https://mseng.pkgs.visualstudio.com/_packaging/VsoMicrosoftExternals/npm/registry/"",
""integrity"": ""sha1-PRT306DRK/NZUaVL07iuqH7nWPg=""
}}
}}
}}";
var packageLockTemplate = string.Format(packageLockJson, componentA.Name, componentA.Version, componentB.Name, componentB.Version);
var packagejson = @"{{
""name"": ""test"",
""version"": ""0.0.0"",
""dependencies"": {{
""{0}"": ""{1}"",
}}
}}";
var packageJsonTemplate = string.Format(packagejson, componentA.Name, componentA.Version);
var (scanResult, componentRecorder) = await this.DetectorTestUtility
.WithFile(this.packageLockJsonFileName, packageLockTemplate, this.packageLockJsonSearchPatterns)
.WithFile(this.packageJsonFileName, packageJsonTemplate, this.packageJsonSearchPattern)
.ExecuteDetectorAsync();
scanResult.ResultCode.Should().Be(ProcessingResultCode.Success);
var detectedComponents = componentRecorder.GetDetectedComponents();
detectedComponents.Should().HaveCount(2);
var componentAId = detectedComponents.First(c => ((NpmComponent)c.Component).Name.Equals(componentA.Name)).Component.Id;
var componentBId = detectedComponents.First(c => ((NpmComponent)c.Component).Name.Equals(componentB.Name)).Component.Id;
var dependencyGraph = componentRecorder.GetDependencyGraphsByLocation().Values.First();
dependencyGraph.GetDependenciesForComponent(componentAId).Should().HaveCount(1);
dependencyGraph.GetDependenciesForComponent(componentAId).Should().Contain(componentBId);
dependencyGraph.GetDependenciesForComponent(componentBId).Should().HaveCount(0);
}
}

Просмотреть файл

@ -269,11 +269,8 @@ public class NpmUtilitiesTests
{
var testCases = new[]
{
("test", "test"),
("@types/test", "@types/test"),
("node_modules/test", "test"),
("node_modules/@types/test", "@types/test"),
("node_modules/root/node_modules/test", "test"),
("test", "test"), ("@types/test", "@types/test"), ("node_modules/test", "test"),
("node_modules/@types/test", "@types/test"), ("node_modules/root/node_modules/test", "test"),
("node_modules/root/node_modules/@types/test", "@types/test"),
("node_modules/rootA/node_modules/rootB/node_modules/test", "test"),
("node_modules/rootA/node_modules/rootB/node_modules/@types/test", "@types/test"),
@ -284,30 +281,4 @@ public class NpmUtilitiesTests
NpmComponentUtilities.GetModuleName(path).Should().Be(expectedModuleName);
}
}
[TestMethod]
public void TestNpmDetector_UpdateLockFileVersion()
{
var envMock = new Mock<IEnvironmentVariableService>();
var loggerMock = new Mock<ILogger>();
var testCases = new[]
{
new { version = 0, env = string.Empty, result = 0 }, new { version = 0, env = "true", result = 0 },
new { version = 1, env = string.Empty, result = 1 }, new { version = 1, env = "true", result = 1 },
new { version = 2, env = string.Empty, result = 2 }, new { version = 2, env = "true", result = 2 },
// 3 should be set as 2 if the env var is not set
new { version = 3, env = string.Empty, result = 2 }, new { version = 3, env = "true", result = 3 },
};
foreach (var testCase in testCases)
{
envMock
.Setup(x => x.GetEnvironmentVariable(NpmComponentUtilities.LockFile3EnvFlag))
.Returns(testCase.env);
NpmComponentUtilities.UpdateLockFileVersion(testCase.version, envMock.Object, loggerMock.Object).Should().Be(testCase.result);
}
}
}