feat(@angular-devkit/build-angular): support karma with esbuild

Adds a new "builderMode" setting for Karma that can be used to switch
between webpack ("browser") and esbuild ("application"). It supports a
third value "detect" that will use the same bundler that's also used for
development builds.

The detect mode is modelled after the logic used for the dev-server builder.

This initial implementation doesn't properly support `--watch` mode or code
coverage.
This commit is contained in:
Jan Martin 2024-09-23 15:58:19 -07:00 committed by Jan Olaf Martin
parent 25c4584210
commit 54594b5abf
16 changed files with 854 additions and 288 deletions

View File

@ -213,6 +213,7 @@ export interface FileReplacement {
export interface KarmaBuilderOptions {
assets?: AssetPattern_2[];
browsers?: Browsers;
builderMode?: BuilderMode;
codeCoverage?: boolean;
codeCoverageExclude?: string[];
exclude?: string[];

View File

@ -0,0 +1,297 @@
/**
* @license
* Copyright Google LLC All Rights Reserved.
*
* Use of this source code is governed by an MIT-style license that can be
* found in the LICENSE file at https://angular.dev/license
*/
import { BuildOutputFileType } from '@angular/build';
import {
ResultFile,
ResultKind,
buildApplicationInternal,
emitFilesToDisk,
purgeStaleBuildCache,
} from '@angular/build/private';
import { BuilderContext, BuilderOutput } from '@angular-devkit/architect';
import { randomUUID } from 'crypto';
import * as fs from 'fs/promises';
import type { Config, ConfigOptions, InlinePluginDef } from 'karma';
import * as path from 'path';
import { Observable, catchError, defaultIfEmpty, from, of, switchMap } from 'rxjs';
import { Configuration } from 'webpack';
import { ExecutionTransformer } from '../../transforms';
import { readTsconfig } from '../../utils/read-tsconfig';
import { OutputHashing } from '../browser-esbuild/schema';
import { findTests } from './find-tests';
import { Schema as KarmaBuilderOptions } from './schema';
class ApplicationBuildError extends Error {
constructor(message: string) {
super(message);
this.name = 'ApplicationBuildError';
}
}
export function execute(
options: KarmaBuilderOptions,
context: BuilderContext,
karmaOptions: ConfigOptions,
transforms: {
webpackConfiguration?: ExecutionTransformer<Configuration>;
// The karma options transform cannot be async without a refactor of the builder implementation
karmaOptions?: (options: ConfigOptions) => ConfigOptions;
} = {},
): Observable<BuilderOutput> {
return from(initializeApplication(options, context, karmaOptions, transforms)).pipe(
switchMap(
([karma, karmaConfig]) =>
new Observable<BuilderOutput>((subscriber) => {
// Complete the observable once the Karma server returns.
const karmaServer = new karma.Server(karmaConfig as Config, (exitCode) => {
subscriber.next({ success: exitCode === 0 });
subscriber.complete();
});
const karmaStart = karmaServer.start();
// Cleanup, signal Karma to exit.
return () => {
void karmaStart.then(() => karmaServer.stop());
};
}),
),
catchError((err) => {
if (err instanceof ApplicationBuildError) {
return of({ success: false, message: err.message });
}
throw err;
}),
defaultIfEmpty({ success: false }),
);
}
async function getProjectSourceRoot(context: BuilderContext): Promise<string> {
// We have already validated that the project name is set before calling this function.
const projectName = context.target?.project;
if (!projectName) {
return context.workspaceRoot;
}
const projectMetadata = await context.getProjectMetadata(projectName);
const sourceRoot = (projectMetadata.sourceRoot ?? projectMetadata.root ?? '') as string;
return path.join(context.workspaceRoot, sourceRoot);
}
async function collectEntrypoints(
options: KarmaBuilderOptions,
context: BuilderContext,
): Promise<[Set<string>, string[]]> {
const projectSourceRoot = await getProjectSourceRoot(context);
// Glob for files to test.
const testFiles = await findTests(
options.include ?? [],
options.exclude ?? [],
context.workspaceRoot,
projectSourceRoot,
);
const entryPoints = new Set([
...testFiles,
'@angular-devkit/build-angular/src/builders/karma/init_test_bed.js',
]);
// Extract `zone.js/testing` to a separate entry point because it needs to be loaded after Jasmine.
const [polyfills, hasZoneTesting] = extractZoneTesting(options.polyfills);
if (hasZoneTesting) {
entryPoints.add('zone.js/testing');
}
const tsConfigPath = path.resolve(context.workspaceRoot, options.tsConfig);
const tsConfig = await readTsconfig(tsConfigPath);
const localizePackageInitEntryPoint = '@angular/localize/init';
const hasLocalizeType = tsConfig.options.types?.some(
(t) => t === '@angular/localize' || t === localizePackageInitEntryPoint,
);
if (hasLocalizeType) {
polyfills.push(localizePackageInitEntryPoint);
}
return [entryPoints, polyfills];
}
async function initializeApplication(
options: KarmaBuilderOptions,
context: BuilderContext,
karmaOptions: ConfigOptions,
transforms: {
webpackConfiguration?: ExecutionTransformer<Configuration>;
karmaOptions?: (options: ConfigOptions) => ConfigOptions;
} = {},
): Promise<[typeof import('karma'), Config & ConfigOptions]> {
if (transforms.webpackConfiguration) {
context.logger.warn(
`This build is using the application builder but transforms.webpackConfiguration was provided. The transform will be ignored.`,
);
}
const testDir = path.join(context.workspaceRoot, 'dist/test-out', randomUUID());
const [karma, [entryPoints, polyfills]] = await Promise.all([
import('karma'),
collectEntrypoints(options, context),
fs.rm(testDir, { recursive: true, force: true }),
]);
const outputPath = testDir;
// Build tests with `application` builder, using test files as entry points.
const buildOutput = await first(
buildApplicationInternal(
{
entryPoints,
tsConfig: options.tsConfig,
outputPath,
aot: false,
index: false,
outputHashing: OutputHashing.None,
optimization: false,
sourceMap: {
scripts: true,
styles: true,
vendor: true,
},
styles: options.styles,
polyfills,
webWorkerTsConfig: options.webWorkerTsConfig,
},
context,
),
);
if (buildOutput.kind === ResultKind.Failure) {
throw new ApplicationBuildError('Build failed');
} else if (buildOutput.kind !== ResultKind.Full) {
throw new ApplicationBuildError(
'A full build result is required from the application builder.',
);
}
// Write test files
await writeTestFiles(buildOutput.files, testDir);
karmaOptions.files ??= [];
karmaOptions.files.push(
// Serve polyfills first.
{ pattern: `${testDir}/polyfills.js`, type: 'module' },
// Allow loading of chunk-* files but don't include them all on load.
{ pattern: `${testDir}/chunk-*.js`, type: 'module', included: false },
// Allow loading of worker-* files but don't include them all on load.
{ pattern: `${testDir}/worker-*.js`, type: 'module', included: false },
// `zone.js/testing`, served but not included on page load.
{ pattern: `${testDir}/testing.js`, type: 'module', included: false },
// Serve remaining JS on page load, these are the test entrypoints.
{ pattern: `${testDir}/*.js`, type: 'module' },
);
if (options.styles?.length) {
// Serve CSS outputs on page load, these are the global styles.
karmaOptions.files.push({ pattern: `${testDir}/*.css`, type: 'css' });
}
const parsedKarmaConfig: Config & ConfigOptions = await karma.config.parseConfig(
options.karmaConfig && path.resolve(context.workspaceRoot, options.karmaConfig),
transforms.karmaOptions ? transforms.karmaOptions(karmaOptions) : karmaOptions,
{ promiseConfig: true, throwErrors: true },
);
// Remove the webpack plugin/framework:
// Alternative would be to make the Karma plugin "smart" but that's a tall order
// with managing unneeded imports etc..
const pluginLengthBefore = (parsedKarmaConfig.plugins ?? []).length;
parsedKarmaConfig.plugins = (parsedKarmaConfig.plugins ?? []).filter(
(plugin: string | InlinePluginDef) => {
if (typeof plugin === 'string') {
return plugin !== 'framework:@angular-devkit/build-angular';
}
return !plugin['framework:@angular-devkit/build-angular'];
},
);
parsedKarmaConfig.frameworks = parsedKarmaConfig.frameworks?.filter(
(framework: string) => framework !== '@angular-devkit/build-angular',
);
const pluginLengthAfter = (parsedKarmaConfig.plugins ?? []).length;
if (pluginLengthBefore !== pluginLengthAfter) {
context.logger.warn(
`Ignoring framework "@angular-devkit/build-angular" from karma config file because it's not compatible with the application builder.`,
);
}
// When using code-coverage, auto-add karma-coverage.
// This was done as part of the karma plugin for webpack.
if (
options.codeCoverage &&
!parsedKarmaConfig.reporters?.some((r: string) => r === 'coverage' || r === 'coverage-istanbul')
) {
parsedKarmaConfig.reporters = (parsedKarmaConfig.reporters ?? []).concat(['coverage']);
}
return [karma, parsedKarmaConfig];
}
export async function writeTestFiles(files: Record<string, ResultFile>, testDir: string) {
const directoryExists = new Set<string>();
// Writes the test related output files to disk and ensures the containing directories are present
await emitFilesToDisk(Object.entries(files), async ([filePath, file]) => {
if (file.type !== BuildOutputFileType.Browser && file.type !== BuildOutputFileType.Media) {
return;
}
const fullFilePath = path.join(testDir, filePath);
// Ensure output subdirectories exist
const fileBasePath = path.dirname(fullFilePath);
if (fileBasePath && !directoryExists.has(fileBasePath)) {
await fs.mkdir(fileBasePath, { recursive: true });
directoryExists.add(fileBasePath);
}
if (file.origin === 'memory') {
// Write file contents
await fs.writeFile(fullFilePath, file.contents);
} else {
// Copy file contents
await fs.copyFile(file.inputPath, fullFilePath, fs.constants.COPYFILE_FICLONE);
}
});
}
function extractZoneTesting(
polyfills: readonly string[] | string | undefined,
): [polyfills: string[], hasZoneTesting: boolean] {
if (typeof polyfills === 'string') {
polyfills = [polyfills];
}
polyfills ??= [];
const polyfillsWithoutZoneTesting = polyfills.filter(
(polyfill) => polyfill !== 'zone.js/testing',
);
const hasZoneTesting = polyfills.length !== polyfillsWithoutZoneTesting.length;
return [polyfillsWithoutZoneTesting, hasZoneTesting];
}
/** Returns the first item yielded by the given generator and cancels the execution. */
async function first<T>(generator: AsyncIterable<T>): Promise<T> {
for await (const value of generator) {
return value;
}
throw new Error('Expected generator to emit at least once.');
}

View File

@ -0,0 +1,170 @@
/**
* @license
* Copyright Google LLC All Rights Reserved.
*
* Use of this source code is governed by an MIT-style license that can be
* found in the LICENSE file at https://angular.dev/license
*/
import { purgeStaleBuildCache } from '@angular/build/private';
import { BuilderContext, BuilderOutput } from '@angular-devkit/architect';
import type { Config, ConfigOptions } from 'karma';
import * as path from 'path';
import { Observable, defaultIfEmpty, from, switchMap } from 'rxjs';
import { Configuration } from 'webpack';
import { getCommonConfig, getStylesConfig } from '../../tools/webpack/configs';
import { ExecutionTransformer } from '../../transforms';
import { generateBrowserWebpackConfigFromContext } from '../../utils/webpack-browser-config';
import { Schema as BrowserBuilderOptions, OutputHashing } from '../browser/schema';
import { FindTestsPlugin } from './find-tests-plugin';
import { Schema as KarmaBuilderOptions } from './schema';
export type KarmaConfigOptions = ConfigOptions & {
buildWebpack?: unknown;
configFile?: string;
};
export function execute(
options: KarmaBuilderOptions,
context: BuilderContext,
karmaOptions: KarmaConfigOptions,
transforms: {
webpackConfiguration?: ExecutionTransformer<Configuration>;
// The karma options transform cannot be async without a refactor of the builder implementation
karmaOptions?: (options: KarmaConfigOptions) => KarmaConfigOptions;
} = {},
): Observable<BuilderOutput> {
return from(initializeBrowser(options, context)).pipe(
switchMap(async ([karma, webpackConfig]) => {
const projectName = context.target?.project;
if (!projectName) {
throw new Error(`The 'karma' builder requires a target to be specified.`);
}
const projectMetadata = await context.getProjectMetadata(projectName);
const sourceRoot = (projectMetadata.sourceRoot ?? projectMetadata.root ?? '') as string;
if (!options.main) {
webpackConfig.entry ??= {};
if (typeof webpackConfig.entry === 'object' && !Array.isArray(webpackConfig.entry)) {
if (Array.isArray(webpackConfig.entry['main'])) {
webpackConfig.entry['main'].push(getBuiltInMainFile());
} else {
webpackConfig.entry['main'] = [getBuiltInMainFile()];
}
}
}
webpackConfig.plugins ??= [];
webpackConfig.plugins.push(
new FindTestsPlugin({
include: options.include,
exclude: options.exclude,
workspaceRoot: context.workspaceRoot,
projectSourceRoot: path.join(context.workspaceRoot, sourceRoot),
}),
);
karmaOptions.buildWebpack = {
options,
webpackConfig,
logger: context.logger,
};
const parsedKarmaConfig = await karma.config.parseConfig(
options.karmaConfig && path.resolve(context.workspaceRoot, options.karmaConfig),
transforms.karmaOptions ? transforms.karmaOptions(karmaOptions) : karmaOptions,
{ promiseConfig: true, throwErrors: true },
);
return [karma, parsedKarmaConfig] as [typeof karma, KarmaConfigOptions];
}),
switchMap(
([karma, karmaConfig]) =>
new Observable<BuilderOutput>((subscriber) => {
// Pass onto Karma to emit BuildEvents.
karmaConfig.buildWebpack ??= {};
if (typeof karmaConfig.buildWebpack === 'object') {
// eslint-disable-next-line @typescript-eslint/no-explicit-any
(karmaConfig.buildWebpack as any).failureCb ??= () =>
subscriber.next({ success: false });
// eslint-disable-next-line @typescript-eslint/no-explicit-any
(karmaConfig.buildWebpack as any).successCb ??= () =>
subscriber.next({ success: true });
}
// Complete the observable once the Karma server returns.
const karmaServer = new karma.Server(karmaConfig as Config, (exitCode) => {
subscriber.next({ success: exitCode === 0 });
subscriber.complete();
});
const karmaStart = karmaServer.start();
// Cleanup, signal Karma to exit.
return () => {
void karmaStart.then(() => karmaServer.stop());
};
}),
),
defaultIfEmpty({ success: false }),
);
}
async function initializeBrowser(
options: KarmaBuilderOptions,
context: BuilderContext,
webpackConfigurationTransformer?: ExecutionTransformer<Configuration>,
): Promise<[typeof import('karma'), Configuration]> {
// Purge old build disk cache.
await purgeStaleBuildCache(context);
const karma = await import('karma');
const { config } = await generateBrowserWebpackConfigFromContext(
// only two properties are missing:
// * `outputPath` which is fixed for tests
// * `budgets` which might be incorrect due to extra dev libs
{
...(options as unknown as BrowserBuilderOptions),
outputPath: '',
budgets: undefined,
optimization: false,
buildOptimizer: false,
aot: false,
vendorChunk: true,
namedChunks: true,
extractLicenses: false,
outputHashing: OutputHashing.None,
// The webpack tier owns the watch behavior so we want to force it in the config.
// When not in watch mode, webpack-dev-middleware will call `compiler.watch` anyway.
// https://github.com/webpack/webpack-dev-middleware/blob/698c9ae5e9bb9a013985add6189ff21c1a1ec185/src/index.js#L65
// https://github.com/webpack/webpack/blob/cde1b73e12eb8a77eb9ba42e7920c9ec5d29c2c9/lib/Compiler.js#L379-L388
watch: true,
},
context,
(wco) => [getCommonConfig(wco), getStylesConfig(wco)],
);
return [karma, (await webpackConfigurationTransformer?.(config)) ?? config];
}
function getBuiltInMainFile(): string {
const content = Buffer.from(
`
import { getTestBed } from '@angular/core/testing';
import {
BrowserDynamicTestingModule,
platformBrowserDynamicTesting,
} from '@angular/platform-browser-dynamic/testing';
// Initialize the Angular testing environment.
getTestBed().initTestEnvironment(BrowserDynamicTestingModule, platformBrowserDynamicTesting(), {
errorOnUnknownElements: true,
errorOnUnknownProperties: true
});
`,
).toString('base64');
return `ng-virtual-main.js!=!data:text/javascript;base64,${content}`;
}

View File

@ -7,12 +7,11 @@
*/
import assert from 'assert';
import glob, { isDynamicPattern } from 'fast-glob';
import { PathLike, constants, promises as fs } from 'fs';
import { pluginName } from 'mini-css-extract-plugin';
import { basename, dirname, extname, join, relative } from 'path';
import type { Compilation, Compiler } from 'webpack';
import { findTests } from './find-tests';
/**
* The name of the plugin provided to Webpack when tapping Webpack compiler hooks.
*/
@ -71,105 +70,3 @@ export class FindTestsPlugin {
});
}
}
// go through all patterns and find unique list of files
async function findTests(
include: string[],
exclude: string[],
workspaceRoot: string,
projectSourceRoot: string,
): Promise<string[]> {
const matchingTestsPromises = include.map((pattern) =>
findMatchingTests(pattern, exclude, workspaceRoot, projectSourceRoot),
);
const files = await Promise.all(matchingTestsPromises);
// Unique file names
return [...new Set(files.flat())];
}
const normalizePath = (path: string): string => path.replace(/\\/g, '/');
const removeLeadingSlash = (pattern: string): string => {
if (pattern.charAt(0) === '/') {
return pattern.substring(1);
}
return pattern;
};
const removeRelativeRoot = (path: string, root: string): string => {
if (path.startsWith(root)) {
return path.substring(root.length);
}
return path;
};
async function findMatchingTests(
pattern: string,
ignore: string[],
workspaceRoot: string,
projectSourceRoot: string,
): Promise<string[]> {
// normalize pattern, glob lib only accepts forward slashes
let normalizedPattern = normalizePath(pattern);
normalizedPattern = removeLeadingSlash(normalizedPattern);
const relativeProjectRoot = normalizePath(relative(workspaceRoot, projectSourceRoot) + '/');
// remove relativeProjectRoot to support relative paths from root
// such paths are easy to get when running scripts via IDEs
normalizedPattern = removeRelativeRoot(normalizedPattern, relativeProjectRoot);
// special logic when pattern does not look like a glob
if (!isDynamicPattern(normalizedPattern)) {
if (await isDirectory(join(projectSourceRoot, normalizedPattern))) {
normalizedPattern = `${normalizedPattern}/**/*.spec.@(ts|tsx)`;
} else {
// see if matching spec file exists
const fileExt = extname(normalizedPattern);
// Replace extension to `.spec.ext`. Example: `src/app/app.component.ts`-> `src/app/app.component.spec.ts`
const potentialSpec = join(
projectSourceRoot,
dirname(normalizedPattern),
`${basename(normalizedPattern, fileExt)}.spec${fileExt}`,
);
if (await exists(potentialSpec)) {
return [potentialSpec];
}
}
}
// normalize the patterns in the ignore list
const normalizedIgnorePatternList = ignore.map((pattern: string) =>
removeRelativeRoot(removeLeadingSlash(normalizePath(pattern)), relativeProjectRoot),
);
return glob(normalizedPattern, {
cwd: projectSourceRoot,
absolute: true,
ignore: ['**/node_modules/**', ...normalizedIgnorePatternList],
});
}
async function isDirectory(path: PathLike): Promise<boolean> {
try {
const stats = await fs.stat(path);
return stats.isDirectory();
} catch {
return false;
}
}
async function exists(path: PathLike): Promise<boolean> {
try {
await fs.access(path, constants.F_OK);
return true;
} catch {
return false;
}
}

View File

@ -0,0 +1,113 @@
/**
* @license
* Copyright Google LLC All Rights Reserved.
*
* Use of this source code is governed by an MIT-style license that can be
* found in the LICENSE file at https://angular.dev/license
*/
import glob, { isDynamicPattern } from 'fast-glob';
import { PathLike, constants, promises as fs } from 'fs';
import { basename, dirname, extname, join, relative } from 'path';
/* Go through all patterns and find unique list of files */
export async function findTests(
include: string[],
exclude: string[],
workspaceRoot: string,
projectSourceRoot: string,
): Promise<string[]> {
const matchingTestsPromises = include.map((pattern) =>
findMatchingTests(pattern, exclude, workspaceRoot, projectSourceRoot),
);
const files = await Promise.all(matchingTestsPromises);
// Unique file names
return [...new Set(files.flat())];
}
const normalizePath = (path: string): string => path.replace(/\\/g, '/');
const removeLeadingSlash = (pattern: string): string => {
if (pattern.charAt(0) === '/') {
return pattern.substring(1);
}
return pattern;
};
const removeRelativeRoot = (path: string, root: string): string => {
if (path.startsWith(root)) {
return path.substring(root.length);
}
return path;
};
async function findMatchingTests(
pattern: string,
ignore: string[],
workspaceRoot: string,
projectSourceRoot: string,
): Promise<string[]> {
// normalize pattern, glob lib only accepts forward slashes
let normalizedPattern = normalizePath(pattern);
normalizedPattern = removeLeadingSlash(normalizedPattern);
const relativeProjectRoot = normalizePath(relative(workspaceRoot, projectSourceRoot) + '/');
// remove relativeProjectRoot to support relative paths from root
// such paths are easy to get when running scripts via IDEs
normalizedPattern = removeRelativeRoot(normalizedPattern, relativeProjectRoot);
// special logic when pattern does not look like a glob
if (!isDynamicPattern(normalizedPattern)) {
if (await isDirectory(join(projectSourceRoot, normalizedPattern))) {
normalizedPattern = `${normalizedPattern}/**/*.spec.@(ts|tsx)`;
} else {
// see if matching spec file exists
const fileExt = extname(normalizedPattern);
// Replace extension to `.spec.ext`. Example: `src/app/app.component.ts`-> `src/app/app.component.spec.ts`
const potentialSpec = join(
projectSourceRoot,
dirname(normalizedPattern),
`${basename(normalizedPattern, fileExt)}.spec${fileExt}`,
);
if (await exists(potentialSpec)) {
return [potentialSpec];
}
}
}
// normalize the patterns in the ignore list
const normalizedIgnorePatternList = ignore.map((pattern: string) =>
removeRelativeRoot(removeLeadingSlash(normalizePath(pattern)), relativeProjectRoot),
);
return glob(normalizedPattern, {
cwd: projectSourceRoot,
absolute: true,
ignore: ['**/node_modules/**', ...normalizedIgnorePatternList],
});
}
async function isDirectory(path: PathLike): Promise<boolean> {
try {
const stats = await fs.stat(path);
return stats.isDirectory();
} catch {
return false;
}
}
async function exists(path: PathLike): Promise<boolean> {
try {
await fs.access(path, constants.F_OK);
return true;
} catch {
return false;
}
}

View File

@ -6,64 +6,27 @@
* found in the LICENSE file at https://angular.dev/license
*/
import { assertCompatibleAngularVersion, purgeStaleBuildCache } from '@angular/build/private';
import { BuilderContext, BuilderOutput, createBuilder } from '@angular-devkit/architect';
import { assertCompatibleAngularVersion } from '@angular/build/private';
import {
BuilderContext,
BuilderOutput,
createBuilder,
targetFromTargetString,
} from '@angular-devkit/architect';
import { strings } from '@angular-devkit/core';
import type { Config, ConfigOptions } from 'karma';
import type { ConfigOptions } from 'karma';
import { createRequire } from 'module';
import * as path from 'path';
import { Observable, defaultIfEmpty, from, switchMap } from 'rxjs';
import { Observable, from, mergeMap } from 'rxjs';
import { Configuration } from 'webpack';
import { getCommonConfig, getStylesConfig } from '../../tools/webpack/configs';
import { ExecutionTransformer } from '../../transforms';
import { generateBrowserWebpackConfigFromContext } from '../../utils/webpack-browser-config';
import { Schema as BrowserBuilderOptions, OutputHashing } from '../browser/schema';
import { FindTestsPlugin } from './find-tests-plugin';
import { Schema as KarmaBuilderOptions } from './schema';
import { BuilderMode, Schema as KarmaBuilderOptions } from './schema';
export type KarmaConfigOptions = ConfigOptions & {
buildWebpack?: unknown;
configFile?: string;
};
async function initialize(
options: KarmaBuilderOptions,
context: BuilderContext,
webpackConfigurationTransformer?: ExecutionTransformer<Configuration>,
): Promise<[typeof import('karma'), Configuration]> {
// Purge old build disk cache.
await purgeStaleBuildCache(context);
const { config } = await generateBrowserWebpackConfigFromContext(
// only two properties are missing:
// * `outputPath` which is fixed for tests
// * `budgets` which might be incorrect due to extra dev libs
{
...(options as unknown as BrowserBuilderOptions),
outputPath: '',
budgets: undefined,
optimization: false,
buildOptimizer: false,
aot: false,
vendorChunk: true,
namedChunks: true,
extractLicenses: false,
outputHashing: OutputHashing.None,
// The webpack tier owns the watch behavior so we want to force it in the config.
// When not in watch mode, webpack-dev-middleware will call `compiler.watch` anyway.
// https://github.com/webpack/webpack-dev-middleware/blob/698c9ae5e9bb9a013985add6189ff21c1a1ec185/src/index.js#L65
// https://github.com/webpack/webpack/blob/cde1b73e12eb8a77eb9ba42e7920c9ec5d29c2c9/lib/Compiler.js#L379-L388
watch: true,
},
context,
(wco) => [getCommonConfig(wco), getStylesConfig(wco)],
);
const karma = await import('karma');
return [karma, (await webpackConfigurationTransformer?.(config)) ?? config];
}
/**
* @experimental Direct usage of this function is considered experimental.
*/
@ -79,122 +42,68 @@ export function execute(
// Check Angular version.
assertCompatibleAngularVersion(context.workspaceRoot);
return from(getExecuteWithBuilder(options, context)).pipe(
mergeMap(([useEsbuild, executeWithBuilder]) => {
const karmaOptions = getBaseKarmaOptions(options, context, useEsbuild);
return executeWithBuilder.execute(options, context, karmaOptions, transforms);
}),
);
}
function getBaseKarmaOptions(
options: KarmaBuilderOptions,
context: BuilderContext,
useEsbuild: boolean,
): KarmaConfigOptions {
let singleRun: boolean | undefined;
if (options.watch !== undefined) {
singleRun = !options.watch;
}
return from(initialize(options, context, transforms.webpackConfiguration)).pipe(
switchMap(async ([karma, webpackConfig]) => {
// Determine project name from builder context target
const projectName = context.target?.project;
if (!projectName) {
throw new Error(`The 'karma' builder requires a target to be specified.`);
}
// Determine project name from builder context target
const projectName = context.target?.project;
if (!projectName) {
throw new Error(`The 'karma' builder requires a target to be specified.`);
}
const karmaOptions: KarmaConfigOptions = options.karmaConfig
? {}
: getBuiltInKarmaConfig(context.workspaceRoot, projectName);
const karmaOptions: KarmaConfigOptions = options.karmaConfig
? {}
: getBuiltInKarmaConfig(context.workspaceRoot, projectName, useEsbuild);
karmaOptions.singleRun = singleRun;
karmaOptions.singleRun = singleRun;
// Workaround https://github.com/angular/angular-cli/issues/28271, by clearing context by default
// for single run executions. Not clearing context for multi-run (watched) builds allows the
// Jasmine Spec Runner to be visible in the browser after test execution.
karmaOptions.client ??= {};
karmaOptions.client.clearContext ??= singleRun ?? false; // `singleRun` defaults to `false` per Karma docs.
// Workaround https://github.com/angular/angular-cli/issues/28271, by clearing context by default
// for single run executions. Not clearing context for multi-run (watched) builds allows the
// Jasmine Spec Runner to be visible in the browser after test execution.
karmaOptions.client ??= {};
karmaOptions.client.clearContext ??= singleRun ?? false; // `singleRun` defaults to `false` per Karma docs.
// Convert browsers from a string to an array
if (typeof options.browsers === 'string' && options.browsers) {
karmaOptions.browsers = options.browsers.split(',');
} else if (options.browsers === false) {
karmaOptions.browsers = [];
}
// Convert browsers from a string to an array
if (typeof options.browsers === 'string' && options.browsers) {
karmaOptions.browsers = options.browsers.split(',');
} else if (options.browsers === false) {
karmaOptions.browsers = [];
}
if (options.reporters) {
// Split along commas to make it more natural, and remove empty strings.
const reporters = options.reporters
.reduce<string[]>((acc, curr) => acc.concat(curr.split(',')), [])
.filter((x) => !!x);
if (options.reporters) {
// Split along commas to make it more natural, and remove empty strings.
const reporters = options.reporters
.reduce<string[]>((acc, curr) => acc.concat(curr.split(',')), [])
.filter((x) => !!x);
if (reporters.length > 0) {
karmaOptions.reporters = reporters;
}
}
if (reporters.length > 0) {
karmaOptions.reporters = reporters;
}
}
if (!options.main) {
webpackConfig.entry ??= {};
if (typeof webpackConfig.entry === 'object' && !Array.isArray(webpackConfig.entry)) {
if (Array.isArray(webpackConfig.entry['main'])) {
webpackConfig.entry['main'].push(getBuiltInMainFile());
} else {
webpackConfig.entry['main'] = [getBuiltInMainFile()];
}
}
}
const projectMetadata = await context.getProjectMetadata(projectName);
const sourceRoot = (projectMetadata.sourceRoot ?? projectMetadata.root ?? '') as string;
webpackConfig.plugins ??= [];
webpackConfig.plugins.push(
new FindTestsPlugin({
include: options.include,
exclude: options.exclude,
workspaceRoot: context.workspaceRoot,
projectSourceRoot: path.join(context.workspaceRoot, sourceRoot),
}),
);
karmaOptions.buildWebpack = {
options,
webpackConfig,
logger: context.logger,
};
const parsedKarmaConfig = await karma.config.parseConfig(
options.karmaConfig && path.resolve(context.workspaceRoot, options.karmaConfig),
transforms.karmaOptions ? transforms.karmaOptions(karmaOptions) : karmaOptions,
{ promiseConfig: true, throwErrors: true },
);
return [karma, parsedKarmaConfig] as [typeof karma, KarmaConfigOptions];
}),
switchMap(
([karma, karmaConfig]) =>
new Observable<BuilderOutput>((subscriber) => {
// Pass onto Karma to emit BuildEvents.
karmaConfig.buildWebpack ??= {};
if (typeof karmaConfig.buildWebpack === 'object') {
// eslint-disable-next-line @typescript-eslint/no-explicit-any
(karmaConfig.buildWebpack as any).failureCb ??= () =>
subscriber.next({ success: false });
// eslint-disable-next-line @typescript-eslint/no-explicit-any
(karmaConfig.buildWebpack as any).successCb ??= () =>
subscriber.next({ success: true });
}
// Complete the observable once the Karma server returns.
const karmaServer = new karma.Server(karmaConfig as Config, (exitCode) => {
subscriber.next({ success: exitCode === 0 });
subscriber.complete();
});
const karmaStart = karmaServer.start();
// Cleanup, signal Karma to exit.
return () => {
void karmaStart.then(() => karmaServer.stop());
};
}),
),
defaultIfEmpty({ success: false }),
);
return karmaOptions;
}
function getBuiltInKarmaConfig(
workspaceRoot: string,
projectName: string,
useEsbuild: boolean,
): ConfigOptions & Record<string, unknown> {
let coverageFolderName = projectName.charAt(0) === '@' ? projectName.slice(1) : projectName;
if (/[A-Z]/.test(coverageFolderName)) {
@ -206,13 +115,13 @@ function getBuiltInKarmaConfig(
// Any changes to the config here need to be synced to: packages/schematics/angular/config/files/karma.conf.js.template
return {
basePath: '',
frameworks: ['jasmine', '@angular-devkit/build-angular'],
frameworks: ['jasmine', ...(useEsbuild ? [] : ['@angular-devkit/build-angular'])],
plugins: [
'karma-jasmine',
'karma-chrome-launcher',
'karma-jasmine-html-reporter',
'karma-coverage',
'@angular-devkit/build-angular/plugins/karma',
...(useEsbuild ? [] : ['@angular-devkit/build-angular/plugins/karma']),
].map((p) => workspaceRootRequire(p)),
jasmineHtmlReporter: {
suppressAll: true, // removes the duplicated traces
@ -243,22 +152,62 @@ function getBuiltInKarmaConfig(
export type { KarmaBuilderOptions };
export default createBuilder<Record<string, string> & KarmaBuilderOptions>(execute);
function getBuiltInMainFile(): string {
const content = Buffer.from(
`
import { getTestBed } from '@angular/core/testing';
import {
BrowserDynamicTestingModule,
platformBrowserDynamicTesting,
} from '@angular/platform-browser-dynamic/testing';
async function getExecuteWithBuilder(
options: KarmaBuilderOptions,
context: BuilderContext,
): Promise<[boolean, typeof import('./application_builder') | typeof import('./browser_builder')]> {
const useEsbuild = await checkForEsbuild(options, context);
const executeWithBuilderModule = useEsbuild
? import('./application_builder')
: import('./browser_builder');
// Initialize the Angular testing environment.
getTestBed().initTestEnvironment(BrowserDynamicTestingModule, platformBrowserDynamicTesting(), {
errorOnUnknownElements: true,
errorOnUnknownProperties: true
});
`,
).toString('base64');
return `ng-virtual-main.js!=!data:text/javascript;base64,${content}`;
return [useEsbuild, await executeWithBuilderModule];
}
async function checkForEsbuild(
options: KarmaBuilderOptions,
context: BuilderContext,
): Promise<boolean> {
if (options.builderMode !== BuilderMode.Detect) {
return options.builderMode === BuilderMode.Application;
}
// Look up the current project's build target using a development configuration.
const buildTargetSpecifier = `::development`;
const buildTarget = targetFromTargetString(
buildTargetSpecifier,
context.target?.project,
'build',
);
try {
const developmentBuilderName = await context.getBuilderNameForTarget(buildTarget);
return isEsbuildBased(developmentBuilderName);
} catch (e) {
if (!(e instanceof Error) || e.message !== 'Project target does not exist.') {
throw e;
}
// If we can't find a development builder, we can't use 'detect'.
throw new Error(
'Failed to detect the builder used by the application. Please set builderMode explicitly.',
);
}
}
function isEsbuildBased(
builderName: string,
): builderName is
| '@angular/build:application'
| '@angular-devkit/build-angular:application'
| '@angular-devkit/build-angular:browser-esbuild' {
if (
builderName === '@angular/build:application' ||
builderName === '@angular-devkit/build-angular:application' ||
builderName === '@angular-devkit/build-angular:browser-esbuild'
) {
return true;
}
return false;
}

View File

@ -0,0 +1,19 @@
/**
* @license
* Copyright Google LLC All Rights Reserved.
*
* Use of this source code is governed by an MIT-style license that can be
* found in the LICENSE file at https://angular.dev/license
*/
import { getTestBed } from '@angular/core/testing';
import {
BrowserDynamicTestingModule,
platformBrowserDynamicTesting,
} from '@angular/platform-browser-dynamic/testing';
// Initialize the Angular testing environment.
getTestBed().initTestEnvironment(BrowserDynamicTestingModule, platformBrowserDynamicTesting(), {
errorOnUnknownElements: true,
errorOnUnknownProperties: true,
});

View File

@ -267,6 +267,12 @@
"type": "string"
}
},
"builderMode": {
"type": "string",
"description": "Determines how to build the code under test. If set to 'detect', attempts to follow the development builder.",
"enum": ["detect", "browser", "application"],
"default": "browser"
},
"webWorkerTsConfig": {
"type": "string",
"description": "TypeScript configuration for Web Worker modules."

View File

@ -23,6 +23,12 @@ const coveragePath = 'coverage/lcov.info';
describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget, isApplicationBuilder) => {
describe('Behavior: "codeCoverage"', () => {
if (isApplicationBuilder) {
beforeEach(() => {
pending('Code coverage not implemented yet for application builder');
});
}
beforeEach(() => {
setupTarget(harness);
});

View File

@ -10,8 +10,14 @@ import { concatMap, count, debounceTime, take, timeout } from 'rxjs';
import { execute } from '../../index';
import { BASE_OPTIONS, KARMA_BUILDER_INFO, describeKarmaBuilder } from '../setup';
describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget) => {
describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget, isApplicationBuilder) => {
describe('Behavior: "Rebuilds"', () => {
if (isApplicationBuilder) {
beforeEach(() => {
pending('--watch not implemented yet for application builder');
});
}
beforeEach(() => {
setupTarget(harness);
});

View File

@ -0,0 +1,74 @@
/**
* @license
* Copyright Google LLC All Rights Reserved.
*
* Use of this source code is governed by an MIT-style license that can be
* found in the LICENSE file at https://angular.dev/license
*/
import { execute } from '../../index';
import { BASE_OPTIONS, KARMA_BUILDER_INFO, describeKarmaBuilder } from '../setup';
import { BuilderMode } from '../../schema';
const ESBUILD_LOG_TEXT = 'Application bundle generation complete.';
describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget, isApplicationTarget) => {
describe('option: "builderMode"', () => {
beforeEach(() => {
setupTarget(harness);
});
it('"application" always uses esbuild', async () => {
harness.useTarget('test', {
...BASE_OPTIONS,
builderMode: BuilderMode.Application,
});
const { result, logs } = await harness.executeOnce();
expect(result?.success).toBeTrue();
expect(logs).toContain(
jasmine.objectContaining({
message: jasmine.stringMatching(ESBUILD_LOG_TEXT),
}),
);
});
it('"browser" always uses webpack', async () => {
harness.useTarget('test', {
...BASE_OPTIONS,
builderMode: BuilderMode.Browser,
});
const { result, logs } = await harness.executeOnce();
expect(result?.success).toBeTrue();
expect(logs).not.toContain(
jasmine.objectContaining({
message: jasmine.stringMatching(ESBUILD_LOG_TEXT),
}),
);
});
it('"detect" follows configuration of the development builder', async () => {
harness.useTarget('test', {
...BASE_OPTIONS,
builderMode: BuilderMode.Detect,
});
const { result, logs } = await harness.executeOnce();
expect(result?.success).toBeTrue();
if (isApplicationTarget) {
expect(logs).toContain(
jasmine.objectContaining({
message: jasmine.stringMatching(ESBUILD_LOG_TEXT),
}),
);
} else {
expect(logs).not.toContain(
jasmine.objectContaining({
message: jasmine.stringMatching(ESBUILD_LOG_TEXT),
}),
);
}
});
});
});

View File

@ -18,8 +18,14 @@ import { BASE_OPTIONS, KARMA_BUILDER_INFO, describeKarmaBuilder } from '../setup
const coveragePath = 'coverage/lcov.info';
describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget) => {
describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget, isApplicationBuilder) => {
describe('Option: "codeCoverageExclude"', () => {
if (isApplicationBuilder) {
beforeEach(() => {
pending('Code coverage not implemented yet for application builder');
});
}
beforeEach(() => {
setupTarget(harness);
});

View File

@ -19,8 +19,14 @@ import { BASE_OPTIONS, KARMA_BUILDER_INFO, describeKarmaBuilder } from '../setup
const coveragePath = 'coverage/lcov.info';
describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget) => {
describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget, isApplicationBuilder) => {
describe('Option: "codeCoverage"', () => {
if (isApplicationBuilder) {
beforeEach(() => {
pending('Code coverage not implemented yet for application builder');
});
}
beforeEach(() => {
setupTarget(harness);
});

View File

@ -134,7 +134,9 @@ describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget) => {
expect(logs).toContain(
jasmine.objectContaining({
level: 'error',
message: jasmine.stringMatching(`Can't resolve 'src/test-style-a.css'`),
message: jasmine.stringMatching(
/(Can't|Could not) resolve ['"]src\/test-style-a.css['"]/,
),
}),
);
});

View File

@ -9,7 +9,7 @@
import { execute } from '../../index';
import { BASE_OPTIONS, KARMA_BUILDER_INFO, describeKarmaBuilder } from '../setup';
describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget) => {
describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget, isApplicationBuilder) => {
describe('Option: "webWorkerTsConfig"', () => {
beforeEach(() => {
setupTarget(harness);
@ -69,15 +69,27 @@ describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget) => {
});
});
it(`should not parse web workers when "webWorkerTsConfig" is not set or set to undefined.`, async () => {
harness.useTarget('test', {
...BASE_OPTIONS,
webWorkerTsConfig: undefined,
});
// Web workers work with the application builder _without_ setting webWorkerTsConfig.
if (isApplicationBuilder) {
it(`should parse web workers when "webWorkerTsConfig" is not set or set to undefined.`, async () => {
harness.useTarget('test', {
...BASE_OPTIONS,
webWorkerTsConfig: undefined,
});
await harness.writeFile(
'./src/app/app.component.spec.ts',
`
const { result } = await harness.executeOnce();
expect(result?.success).toBeTrue();
});
} else {
it(`should not parse web workers when "webWorkerTsConfig" is not set or set to undefined.`, async () => {
harness.useTarget('test', {
...BASE_OPTIONS,
webWorkerTsConfig: undefined,
});
await harness.writeFile(
'./src/app/app.component.spec.ts',
`
import { TestBed } from '@angular/core/testing';
import { AppComponent } from './app.component';
@ -91,11 +103,12 @@ describeKarmaBuilder(execute, KARMA_BUILDER_INFO, (harness, setupTarget) => {
.toThrowError(/Failed to construct 'Worker'/);
});
});`,
);
);
const { result } = await harness.executeOnce();
expect(result?.success).toBeTrue();
});
const { result } = await harness.executeOnce();
expect(result?.success).toBeTrue();
});
}
it(`should parse web workers when "webWorkerTsConfig" is set.`, async () => {
harness.useTarget('test', {

View File

@ -6,7 +6,7 @@
* found in the LICENSE file at https://angular.dev/license
*/
import { Schema } from '../schema';
import { BuilderMode, Schema } from '../schema';
import { BuilderHandlerFn } from '@angular-devkit/architect';
import { json } from '@angular-devkit/core';
import { ApplicationBuilderOptions as ApplicationSchema, buildApplication } from '@angular/build';
@ -41,6 +41,7 @@ export const BASE_OPTIONS = Object.freeze<Schema>({
browsers: 'ChromeHeadlessCI',
progress: false,
watch: false,
builderMode: BuilderMode.Detect,
});
const optionSchemaCache = new Map<string, json.schema.JsonSchema>();