diff --git a/learning/tour-of-beam/frontend/lib/locator.dart b/learning/tour-of-beam/frontend/lib/locator.dart index 46095e92948a9..fedfa528c05a8 100644 --- a/learning/tour-of-beam/frontend/lib/locator.dart +++ b/learning/tour-of-beam/frontend/lib/locator.dart @@ -57,7 +57,6 @@ Future _initializeRepositories() async { final exampleClient = GrpcExampleClient(url: routerUrl); GetIt.instance.registerSingleton(codeClient); - GetIt.instance.registerSingleton(CodeRepository(client: codeClient)); GetIt.instance.registerSingleton(exampleClient); GetIt.instance.registerSingleton(ExampleRepository(client: exampleClient)); } diff --git a/learning/tour-of-beam/frontend/lib/pages/tour/state.dart b/learning/tour-of-beam/frontend/lib/pages/tour/state.dart index a228d3a0621a3..00d2f2fcddc03 100644 --- a/learning/tour-of-beam/frontend/lib/pages/tour/state.dart +++ b/learning/tour-of-beam/frontend/lib/pages/tour/state.dart @@ -351,7 +351,7 @@ class TourNotifier extends ChangeNotifier with PageStateMixin { static PlaygroundController _createPlaygroundController(String initialSdkId) { final playgroundController = PlaygroundController( - codeRepository: GetIt.instance.get(), + codeClient: GetIt.instance.get(), exampleCache: ExampleCache( exampleRepository: GetIt.instance.get(), ), diff --git a/playground/backend/cmd/server/controller.go b/playground/backend/cmd/server/controller.go index 606e5fd4dcfe2..f788f2e613f4f 100644 --- a/playground/backend/cmd/server/controller.go +++ b/playground/backend/cmd/server/controller.go @@ -69,11 +69,17 @@ type playgroundController struct { // RunCode is running code from requests using a particular SDK // - In case of incorrect sdk returns codes.InvalidArgument +// - In case of exceeded number of parallel jobs returns codes.ResourceExhausted // - In case of error during preparing files/folders returns codes.Internal // - In case of no errors saves playground.Status_STATUS_EXECUTING as cache.Status into cache and sets expiration time // for all cache values which will be saved into cache during processing received code. // Returns id of code processing (pipelineId) func (controller *playgroundController) RunCode(ctx context.Context, info *pb.RunCodeRequest) (*pb.RunCodeResponse, error) { + // check if we can take a new RunCode request + if !utils.CheckNumOfTheParallelJobs(controller.env.ApplicationEnvs.WorkingDir(), controller.env.BeamSdkEnvs.NumOfParallelJobs()) { + logger.Warnf("RunCode(): number of parallel jobs is exceeded\n") + return nil, cerrors.ResourceExhaustedError("Error during preparing", "Number of parallel jobs is exceeded") + } // check for correct sdk if info.Sdk != controller.env.BeamSdkEnvs.ApacheBeamSdk { logger.Errorf("RunCode(): request contains incorrect sdk: %s\n", info.Sdk) diff --git a/playground/backend/internal/errors/grpc_errors.go b/playground/backend/internal/errors/grpc_errors.go index b1c744f60685d..59b80267192bb 100644 --- a/playground/backend/internal/errors/grpc_errors.go +++ b/playground/backend/internal/errors/grpc_errors.go @@ -38,3 +38,9 @@ func InternalError(title string, formatMessage string, args ...interface{}) erro message := fmt.Sprintf(formatMessage, args...) return status.Errorf(codes.Internal, "%s: %s", title, message) } + +// ResourceExhaustedError returns error with ResourceExhausted code error and message like "title: message" +func ResourceExhaustedError(title string, formatMessage string, args ...interface{}) error { + message := fmt.Sprintf(formatMessage, args...) + return status.Errorf(codes.ResourceExhausted, "%s: %s", title, message) +} diff --git a/playground/backend/internal/utils/system_utils.go b/playground/backend/internal/utils/system_utils.go index acc2ca29fb2d4..dd631c1b5cdcb 100644 --- a/playground/backend/internal/utils/system_utils.go +++ b/playground/backend/internal/utils/system_utils.go @@ -47,7 +47,7 @@ func GetLivenessFunction() func(writer http.ResponseWriter, request *http.Reques // GetReadinessFunction returns the function that checks the readiness of the server to process a new code processing request func GetReadinessFunction(envs *environment.Environment) func(writer http.ResponseWriter, request *http.Request) { return func(writer http.ResponseWriter, request *http.Request) { - if checkNumOfTheParallelJobs(envs.ApplicationEnvs.WorkingDir(), envs.BeamSdkEnvs.NumOfParallelJobs()) { + if CheckNumOfTheParallelJobs(envs.ApplicationEnvs.WorkingDir(), envs.BeamSdkEnvs.NumOfParallelJobs()) { writer.WriteHeader(http.StatusOK) } else { writer.WriteHeader(http.StatusLocked) @@ -55,13 +55,13 @@ func GetReadinessFunction(envs *environment.Environment) func(writer http.Respon } } -// checkNumOfTheParallelJobs checks the number of currently working code executions. +// CheckNumOfTheParallelJobs checks the number of currently working code executions. // // It counts by the number of the /path/to/workingDir/executableFiles/{pipelineId} folders. // // If it is equals or more than numOfParallelJobs, then returns false. // If it is less than numOfParallelJobs, then returns true. -func checkNumOfTheParallelJobs(workingDir string, numOfParallelJobs int) bool { +func CheckNumOfTheParallelJobs(workingDir string, numOfParallelJobs int) bool { baseFileFolder := filepath.Join(workingDir, executableFiles) _, err := os.Stat(baseFileFolder) if os.IsNotExist(err) { diff --git a/playground/backend/internal/utils/system_utils_test.go b/playground/backend/internal/utils/system_utils_test.go index 1817ba012542d..a68fb8a8e58f4 100644 --- a/playground/backend/internal/utils/system_utils_test.go +++ b/playground/backend/internal/utils/system_utils_test.go @@ -58,7 +58,7 @@ func Test_checkNumOfTheParallelJobs(t *testing.T) { want bool }{ { - // Test case with calling checkNumOfTheParallelJobs when there is no code processing folders. + // Test case with calling CheckNumOfTheParallelJobs when there is no code processing folders. // As a result, want to receive true name: "There is no code processing folder", args: args{ @@ -69,7 +69,7 @@ func Test_checkNumOfTheParallelJobs(t *testing.T) { want: true, }, { - // Test case with calling checkNumOfTheParallelJobs when there is one code processing folder. + // Test case with calling CheckNumOfTheParallelJobs when there is one code processing folder. // As a result, want to receive true name: "Less than needed", args: args{ @@ -85,7 +85,7 @@ func Test_checkNumOfTheParallelJobs(t *testing.T) { want: true, }, { - // Test case with calling checkNumOfTheParallelJobs when the number of the code processing folders is equals numOfParallelJobs. + // Test case with calling CheckNumOfTheParallelJobs when the number of the code processing folders is equals numOfParallelJobs. // As a result, want to receive false name: "There are enough code processing folders", args: args{ @@ -101,7 +101,7 @@ func Test_checkNumOfTheParallelJobs(t *testing.T) { want: false, }, { - // Test case with calling checkNumOfTheParallelJobs when the number of the code processing folders is more than numOfParallelJobs. + // Test case with calling CheckNumOfTheParallelJobs when the number of the code processing folders is more than numOfParallelJobs. // As a result, want to receive false name: "More than needed", args: args{ @@ -120,8 +120,8 @@ func Test_checkNumOfTheParallelJobs(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { tt.prepareFunc() - if got := checkNumOfTheParallelJobs(tt.args.workingDir, tt.args.numOfParallelJobs); got != tt.want { - t.Errorf("checkNumOfTheParallelJobs() = %v, want %v", got, tt.want) + if got := CheckNumOfTheParallelJobs(tt.args.workingDir, tt.args.numOfParallelJobs); got != tt.want { + t.Errorf("CheckNumOfTheParallelJobs() = %v, want %v", got, tt.want) } os.RemoveAll(executableFiles) }) diff --git a/playground/frontend/lib/controllers/factories.dart b/playground/frontend/lib/controllers/factories.dart index 92863c18bb739..3a8e2388790b5 100644 --- a/playground/frontend/lib/controllers/factories.dart +++ b/playground/frontend/lib/controllers/factories.dart @@ -36,7 +36,7 @@ PlaygroundController createPlaygroundController( final controller = PlaygroundController( examplesLoader: ExamplesLoader(), exampleCache: exampleCache, - codeRepository: GetIt.instance.get(), + codeClient: GetIt.instance.get(), ); unawaited(_loadExamples(controller, descriptor)); diff --git a/playground/frontend/lib/locator.dart b/playground/frontend/lib/locator.dart index 8115e2cd88bfb..21496607b6e72 100644 --- a/playground/frontend/lib/locator.dart +++ b/playground/frontend/lib/locator.dart @@ -42,9 +42,6 @@ Future _initializeRepositories() async { ); GetIt.instance.registerSingleton(codeClient); - GetIt.instance.registerSingleton(CodeRepository( - client: codeClient, - )); final exampleClient = GrpcExampleClient(url: routerUrl); GetIt.instance.registerSingleton(exampleClient); diff --git a/playground/frontend/lib/pages/embedded_playground/widgets/embedded_appbar_title.dart b/playground/frontend/lib/pages/embedded_playground/widgets/embedded_appbar_title.dart index 75443f03b5a8b..09b4370f72b17 100644 --- a/playground/frontend/lib/pages/embedded_playground/widgets/embedded_appbar_title.dart +++ b/playground/frontend/lib/pages/embedded_playground/widgets/embedded_appbar_title.dart @@ -44,7 +44,9 @@ class EmbeddedAppBarTitle extends StatelessWidget { icon: SvgPicture.asset(Assets.copy), onPressed: () { final source = controller.source; - Clipboard.setData(ClipboardData(text: source)); + if (source != null) { + Clipboard.setData(ClipboardData(text: source)); + } }, ), ], diff --git a/playground/frontend/playground_components/lib/playground_components.dart b/playground/frontend/playground_components/lib/playground_components.dart index 6599def9d8db6..06959a14d9608 100644 --- a/playground/frontend/playground_components/lib/playground_components.dart +++ b/playground/frontend/playground_components/lib/playground_components.dart @@ -64,7 +64,6 @@ export 'src/playground_components.dart'; export 'src/repositories/backend_urls.dart'; export 'src/repositories/code_client/code_client.dart'; export 'src/repositories/code_client/grpc_code_client.dart'; -export 'src/repositories/code_repository.dart'; export 'src/repositories/example_client/example_client.dart'; export 'src/repositories/example_client/grpc_example_client.dart'; export 'src/repositories/example_repository.dart'; diff --git a/playground/frontend/playground_components/lib/src/controllers/code_runner.dart b/playground/frontend/playground_components/lib/src/controllers/code_runner.dart index ff89d1c1a2e05..68b357b363263 100644 --- a/playground/frontend/playground_components/lib/src/controllers/code_runner.dart +++ b/playground/frontend/playground_components/lib/src/controllers/code_runner.dart @@ -25,14 +25,28 @@ import 'package:flutter/material.dart'; import '../../playground_components.dart'; import '../enums/unread_entry.dart'; +import '../repositories/models/output_response.dart'; +import '../repositories/models/run_code_error.dart'; import '../repositories/models/run_code_request.dart'; +import '../repositories/models/run_code_response.dart'; import '../repositories/models/run_code_result.dart'; import '../util/connectivity_result.dart'; import 'snippet_editing_controller.dart'; import 'unread_controller.dart'; +const kTimeoutErrorText = + 'Pipeline exceeded Playground execution timeout and was terminated. ' + 'We recommend installing Apache Beam ' + 'https://beam.apache.org/get-started/downloads/ ' + 'to try examples without timeout limitation.'; +const kUnknownErrorText = + 'Something went wrong. Please try again later or create a GitHub issue'; +const kProcessingStartedText = 'The processing has been started\n'; +const kProcessingStartedOptionsText = + 'The processing has been started with the pipeline options: '; + class CodeRunner extends ChangeNotifier { - final CodeRepository? _codeRepository; + final CodeClient? codeClient; final ValueGetter _snippetEditingControllerGetter; SnippetEditingController? snippetEditingController; final unreadController = UnreadController(); @@ -40,9 +54,8 @@ class CodeRunner extends ChangeNotifier { CodeRunner({ required ValueGetter snippetEditingControllerGetter, - CodeRepository? codeRepository, - }) : _codeRepository = codeRepository, - _snippetEditingControllerGetter = snippetEditingControllerGetter; + this.codeClient, + }) : _snippetEditingControllerGetter = snippetEditingControllerGetter; RunCodeResult? _result; StreamSubscription? _runSubscription; @@ -64,12 +77,17 @@ class CodeRunner extends ChangeNotifier { _snippetEditingControllerGetter()?.pipelineOptions; RunCodeResult? get result => _result; + DateTime? get runStartDate => _runStartDate; + DateTime? get runStopDate => _runStopDate; + bool get isCodeRunning => !(_result?.isFinished ?? true); String get resultLog => _result?.log ?? ''; + String get resultOutput => _result?.output ?? ''; + String get resultLogOutput => resultLog + resultOutput; bool get isExampleChanged { @@ -78,14 +96,18 @@ class CodeRunner extends ChangeNotifier { // Snapshot of additional analytics data at the time when execution started. Map _analyticsData = const {}; + Map get analyticsData => _analyticsData; bool get canRun => _snippetEditingControllerGetter() != null; + static const _attempts = 6; + static const _attemptInterval = Duration(seconds: 5); + static const _statusCheckInterval = Duration(seconds: 1); + void clearResult() { _eventSnippetContext = null; _setResult(null); - notifyListeners(); } Future reset() async { @@ -96,21 +118,28 @@ class CodeRunner extends ChangeNotifier { _runStopDate = null; _eventSnippetContext = null; _setResult(null); - notifyListeners(); } - void runCode({ - void Function()? onFinish, + Future runCode({ Map analyticsData = const {}, - }) { + }) async { _analyticsData = analyticsData; _runStartDate = DateTime.now(); _runStopDate = null; - notifyListeners(); snippetEditingController = _snippetEditingControllerGetter(); _eventSnippetContext = snippetEditingController!.eventSnippetContext; - final sdk = snippetEditingController!.sdk; + if (!isExampleChanged && + snippetEditingController!.example?.outputs != null) { + await _showPrecompiledResult(); + return; + } + + await _runReal(); + } + + Future _runReal() async { + final sdk = snippetEditingController!.sdk; final parsedPipelineOptions = parsePipelineOptions(snippetEditingController!.pipelineOptions); if (parsedPipelineOptions == null) { @@ -121,37 +150,115 @@ class CodeRunner extends ChangeNotifier { status: RunCodeStatus.compileError, ), ); - _runStopDate = DateTime.now(); - notifyListeners(); return; } - if (!isExampleChanged && - snippetEditingController!.example?.outputs != null) { - unawaited(_showPrecompiledResult()); - } else { - final request = RunCodeRequest( - datasets: snippetEditingController?.example?.datasets ?? [], - files: snippetEditingController!.getFiles(), - sdk: snippetEditingController!.sdk, - pipelineOptions: parsedPipelineOptions, - ); - _runSubscription = _codeRepository?.runCode(request).listen((event) { - _setResult(event); - notifyListeners(); - - if (event.isFinished) { - if (onFinish != null) { - onFinish(); - } - snippetEditingController = null; - _runStopDate = DateTime.now(); + final log = parsedPipelineOptions.isEmpty + ? kProcessingStartedText + // ignore: prefer_interpolation_to_compose_strings + : kProcessingStartedOptionsText + + parsedPipelineOptions.entries + .map((e) => '--${e.key} ${e.value}') + .join(' ') + + '\n'; + + _setResult( + RunCodeResult( + log: log, + sdk: sdk, + status: RunCodeStatus.preparation, + ), + ); + + final request = RunCodeRequest( + datasets: snippetEditingController?.example?.datasets ?? [], + files: snippetEditingController!.getFiles(), + sdk: snippetEditingController!.sdk, + pipelineOptions: parsedPipelineOptions, + ); + + try { + final runResponse = await _startExecution(request); + + if (runResponse == null || _result!.isFinished) { + // Cancelled while trying to start. + final pipelineUuid = runResponse?.pipelineUuid; + if (pipelineUuid != null) { + await codeClient?.cancelExecution(pipelineUuid); } - }); - notifyListeners(); + return; + } + + await Future.delayed(_statusCheckInterval); + + while (!_result!.isFinished) { + final statusResponse = + await codeClient!.checkStatus(runResponse.pipelineUuid); + + final result = await _getPipelineResult( + runResponse.pipelineUuid, + statusResponse.status, + _result!, + ); + + _setResultIfNotFinished(result); + + await Future.delayed(_statusCheckInterval); + } + } on RunCodeError catch (ex) { + _setResult( + RunCodeResult( + errorMessage: ex.message ?? kUnknownErrorText, + output: ex.message ?? kUnknownErrorText, + sdk: request.sdk, + status: RunCodeStatus.unknownError, + ), + ); + } on Exception catch (ex) { + print(ex); // ignore: avoid_print + _setResult( + RunCodeResult( + errorMessage: kUnknownErrorText, + output: kUnknownErrorText, + sdk: request.sdk, + status: RunCodeStatus.unknownError, + ), + ); + } finally { + snippetEditingController = null; } } + Future _startExecution(RunCodeRequest request) async { + Exception? lastException; + + // Attempts to place the job for execution. + // This fails if the backend is overloaded and has not yet scaled up. + for (int attemptsLeft = _attempts; --attemptsLeft >= 0;) { + if (_result!.isFinished) { + return null; // Cancelled while retrying. + } + + try { + return await codeClient!.runCode(request); + } on RunCodeResourceExhaustedError catch (ex) { + lastException = ex; + } + + // ignore: avoid_print + print( + 'Got RunCodeResourceExhaustedError, attempts left: $attemptsLeft.', + ); + if (attemptsLeft > 0) { + // ignore: avoid_print + print('Waiting for $_attemptInterval before retrying.'); + await Future.delayed(_attemptInterval); + } + } + + throw lastException ?? Exception('lastException must be filled above.'); + } + /// Resets the error message text so that on the next rebuild /// of `CodeTextAreaWrapper` it is not picked up and not shown as a toast. // TODO: Listen to this object outside of widgets, @@ -195,13 +302,10 @@ class CodeRunner extends ChangeNotifier { } snippetEditingController = null; - // Awaited cancelling subscription here blocks further method execution. - // TODO: Figure out the reason: https://github.com/apache/beam/issues/25509 - unawaited(_runSubscription?.cancel()); final pipelineUuid = _result?.pipelineUuid ?? ''; if (pipelineUuid.isNotEmpty) { - await _codeRepository?.cancelExecution(pipelineUuid); + await codeClient?.cancelExecution(pipelineUuid); } _setResult( @@ -213,12 +317,9 @@ class CodeRunner extends ChangeNotifier { 'widgets.output.messages.pipelineCancelled'.tr(), output: _result?.output, sdk: sdk, - status: RunCodeStatus.finished, + status: RunCodeStatus.cancelled, ), ); - - _runStopDate = DateTime.now(); - notifyListeners(); } Future _showPrecompiledResult() async { @@ -250,12 +351,21 @@ class CodeRunner extends ChangeNotifier { status: RunCodeStatus.finished, ), ); + } - _runStopDate = DateTime.now(); - notifyListeners(); + void _setResultIfNotFinished(RunCodeResult newValue) { + if (_result?.isFinished ?? true) { + return; + } + _setResult(newValue); } void _setResult(RunCodeResult? newValue) { + // ignore: use_if_null_to_convert_nulls_to_bools + if (_result?.isFinished == false && newValue?.isFinished == true) { + _runStopDate = clock.now(); + } + _result = newValue; if (newValue == null) { @@ -270,5 +380,138 @@ class CodeRunner extends ChangeNotifier { newValue.graph ?? '', ); } + + notifyListeners(); + } + + Future _getPipelineResult( + String pipelineUuid, + RunCodeStatus status, + RunCodeResult prevResult, + ) async { + final prevOutput = prevResult.output ?? ''; + final prevLog = prevResult.log ?? ''; + final prevGraph = prevResult.graph ?? ''; + + switch (status) { + case RunCodeStatus.compileError: + final compileOutput = await codeClient!.getCompileOutput(pipelineUuid); + return RunCodeResult( + graph: prevGraph, + log: prevLog, + output: compileOutput.output, + pipelineUuid: pipelineUuid, + sdk: prevResult.sdk, + status: status, + ); + + case RunCodeStatus.timeout: + return RunCodeResult( + errorMessage: kTimeoutErrorText, + graph: prevGraph, + log: prevLog, + output: kTimeoutErrorText, + pipelineUuid: pipelineUuid, + sdk: prevResult.sdk, + status: status, + ); + + case RunCodeStatus.runError: + final output = await codeClient!.getRunErrorOutput(pipelineUuid); + return RunCodeResult( + graph: prevGraph, + log: prevLog, + output: output.output, + pipelineUuid: pipelineUuid, + sdk: prevResult.sdk, + status: status, + ); + + case RunCodeStatus.validationError: + final output = await codeClient!.getValidationErrorOutput(pipelineUuid); + return RunCodeResult( + graph: prevGraph, + log: prevLog, + output: output.output, + sdk: prevResult.sdk, + status: status, + ); + + case RunCodeStatus.preparationError: + final output = + await codeClient!.getPreparationErrorOutput(pipelineUuid); + return RunCodeResult( + graph: prevGraph, + log: prevLog, + output: output.output, + sdk: prevResult.sdk, + status: status, + ); + + case RunCodeStatus.unknownError: + return RunCodeResult( + errorMessage: kUnknownErrorText, + graph: prevGraph, + log: prevLog, + output: kUnknownErrorText, + pipelineUuid: pipelineUuid, + sdk: prevResult.sdk, + status: status, + ); + + case RunCodeStatus.executing: + final responses = await Future.wait([ + codeClient!.getRunOutput(pipelineUuid), + codeClient!.getLogOutput(pipelineUuid), + prevGraph.isEmpty + ? codeClient!.getGraphOutput(pipelineUuid) + : Future.value(OutputResponse(output: prevGraph)), + ]); + final output = responses[0]; + final log = responses[1]; + final graph = responses[2]; + return RunCodeResult( + graph: graph.output, + log: prevLog + log.output, + output: prevOutput + output.output, + pipelineUuid: pipelineUuid, + sdk: prevResult.sdk, + status: status, + ); + + case RunCodeStatus.cancelled: + case RunCodeStatus.finished: + final responses = await Future.wait([ + codeClient!.getRunOutput(pipelineUuid), + codeClient!.getLogOutput(pipelineUuid), + codeClient!.getRunErrorOutput(pipelineUuid), + prevGraph.isEmpty + ? codeClient!.getGraphOutput(pipelineUuid) + : Future.value(OutputResponse(output: prevGraph)), + ]); + final output = responses[0]; + final log = responses[1]; + final error = responses[2]; + final graph = responses[3]; + return RunCodeResult( + graph: graph.output, + log: prevLog + log.output, + output: prevOutput + output.output + error.output, + pipelineUuid: pipelineUuid, + sdk: prevResult.sdk, + status: status, + ); + + case RunCodeStatus.unspecified: + case RunCodeStatus.preparation: + case RunCodeStatus.compiling: + return RunCodeResult( + graph: prevGraph, + log: prevLog, + pipelineUuid: pipelineUuid, + sdk: prevResult.sdk, + status: status, + ); + } } } diff --git a/playground/frontend/playground_components/lib/src/controllers/playground_controller.dart b/playground/frontend/playground_components/lib/src/controllers/playground_controller.dart index adef2e8ab6c68..4b01020c7956e 100644 --- a/playground/frontend/playground_components/lib/src/controllers/playground_controller.dart +++ b/playground/frontend/playground_components/lib/src/controllers/playground_controller.dart @@ -36,7 +36,7 @@ import '../models/example_loading_descriptors/user_shared_example_loading_descri import '../models/intents.dart'; import '../models/sdk.dart'; import '../models/shortcut.dart'; -import '../repositories/code_repository.dart'; +import '../repositories/code_client/code_client.dart'; import '../services/symbols/loaders/map.dart'; import '../services/symbols/symbols_notifier.dart'; import '../util/logical_keyboard_key.dart'; @@ -67,12 +67,12 @@ class PlaygroundController with ChangeNotifier { PlaygroundController({ required this.exampleCache, required this.examplesLoader, - CodeRepository? codeRepository, + CodeClient? codeClient, }) { examplesLoader.setPlaygroundController(this); codeRunner = CodeRunner( - codeRepository: codeRepository, + codeClient: codeClient, snippetEditingControllerGetter: requireSnippetEditingController, )..addListener(notifyListeners); } diff --git a/playground/frontend/playground_components/lib/src/repositories/code_client/grpc_code_client.dart b/playground/frontend/playground_components/lib/src/repositories/code_client/grpc_code_client.dart index 04f4fb29e2ff6..4392cda9fac86 100644 --- a/playground/frontend/playground_components/lib/src/repositories/code_client/grpc_code_client.dart +++ b/playground/frontend/playground_components/lib/src/repositories/code_client/grpc_code_client.dart @@ -23,6 +23,7 @@ import '../../api/iis_workaround_channel.dart'; import '../../api/v1/api.pbgrpc.dart' as grpc; import '../../models/sdk.dart'; import '../../util/pipeline_options.dart'; +import '../../util/run_with_retry.dart'; import '../dataset_grpc_extension.dart'; import '../models/check_status_response.dart'; import '../models/output_response.dart'; @@ -92,6 +93,14 @@ class GrpcCodeClient implements CodeClient { @override Future checkStatus( String pipelineUuid, + ) async { + return runWithRetry( + () => _checkStatusWithRetry(pipelineUuid), + ); + } + + Future _checkStatusWithRetry( + String pipelineUuid, ) async { final response = await _runSafely( () => _defaultClient.checkStatus( @@ -204,10 +213,12 @@ class GrpcCodeClient implements CodeClient { try { return await invoke(); } on GrpcError catch (error) { - // Internet unavailable issue also returns unknown code error, - // so message was overwritten. - if (error.code == StatusCode.unknown) { - throw RunCodeError(message: 'errors.unknownError'.tr()); + switch (error.code) { + case StatusCode.unknown: + // The default can be misleading for this. + throw RunCodeError(message: 'errors.unknownError'.tr()); + case StatusCode.resourceExhausted: + throw RunCodeResourceExhaustedError(message: error.message); } throw RunCodeError(message: error.message); } on Exception catch (_) { @@ -250,6 +261,7 @@ class GrpcCodeClient implements CodeClient { case grpc.Status.STATUS_EXECUTING: return RunCodeStatus.executing; case grpc.Status.STATUS_CANCELED: + return RunCodeStatus.cancelled; case grpc.Status.STATUS_FINISHED: return RunCodeStatus.finished; case grpc.Status.STATUS_COMPILE_ERROR: diff --git a/playground/frontend/playground_components/lib/src/repositories/code_repository.dart b/playground/frontend/playground_components/lib/src/repositories/code_repository.dart deleted file mode 100644 index bd6a5206ed1b4..0000000000000 --- a/playground/frontend/playground_components/lib/src/repositories/code_repository.dart +++ /dev/null @@ -1,241 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import '../util/run_with_retry.dart'; -import 'code_client/code_client.dart'; -import 'models/output_response.dart'; -import 'models/run_code_error.dart'; -import 'models/run_code_request.dart'; -import 'models/run_code_result.dart'; - -const kPipelineCheckDelay = Duration(seconds: 1); -const kTimeoutErrorText = - 'Pipeline exceeded Playground execution timeout and was terminated. ' - 'We recommend installing Apache Beam ' - 'https://beam.apache.org/get-started/downloads/ ' - 'to try examples without timeout limitation.'; -const kUnknownErrorText = - 'Something went wrong. Please try again later or create a GitHub issue'; -const kProcessingStartedText = 'The processing has been started\n'; -const kProcessingStartedOptionsText = - 'The processing has been started with the pipeline options: '; - -// TODO(alexeyinkin): Rename. This is not a repository but a higher level client. -class CodeRepository { - final CodeClient _client; - - CodeRepository({ - required CodeClient client, - }) : _client = client; - - Stream runCode(RunCodeRequest request) async* { - try { - final log = request.pipelineOptions.isEmpty - ? kProcessingStartedText - // ignore: prefer_interpolation_to_compose_strings - : kProcessingStartedOptionsText + - request.pipelineOptions.entries - .map((e) => '--${e.key} ${e.value}') - .join(' ') + '\n'; - final initResult = RunCodeResult( - log: log, - sdk: request.sdk, - status: RunCodeStatus.preparation, - ); - yield initResult; - - final runCodeResponse = await _client.runCode(request); - final pipelineUuid = runCodeResponse.pipelineUuid; - - yield* _checkPipelineExecution( - pipelineUuid, - prevResult: initResult, - ); - } on RunCodeError catch (error) { - yield RunCodeResult( - errorMessage: error.message ?? kUnknownErrorText, - output: error.message ?? kUnknownErrorText, - sdk: request.sdk, - status: RunCodeStatus.unknownError, - ); - } - } - - Future cancelExecution(String pipelineUuid) { - return _client.cancelExecution(pipelineUuid); - } - - Stream _checkPipelineExecution( - String pipelineUuid, { - required RunCodeResult prevResult, - }) async* { - try { - final statusResponse = await runWithRetry( - () => _client.checkStatus(pipelineUuid), - ); - final result = await _getPipelineResult( - pipelineUuid, - statusResponse.status, - prevResult, - ); - yield result; - if (!result.isFinished) { - await Future.delayed(kPipelineCheckDelay); - yield* _checkPipelineExecution( - pipelineUuid, - prevResult: result, - ); - } - } on RunCodeError catch (error) { - yield RunCodeResult( - errorMessage: error.message ?? kUnknownErrorText, - output: error.message ?? kUnknownErrorText, - pipelineUuid: prevResult.pipelineUuid, - sdk: prevResult.sdk, - status: RunCodeStatus.unknownError, - ); - } - } - - Future _getPipelineResult( - String pipelineUuid, - RunCodeStatus status, - RunCodeResult prevResult, - ) async { - final prevOutput = prevResult.output ?? ''; - final prevLog = prevResult.log ?? ''; - final prevGraph = prevResult.graph ?? ''; - - switch (status) { - case RunCodeStatus.compileError: - final compileOutput = await _client.getCompileOutput(pipelineUuid); - return RunCodeResult( - graph: prevGraph, - log: prevLog, - output: compileOutput.output, - pipelineUuid: pipelineUuid, - sdk: prevResult.sdk, - status: status, - ); - - case RunCodeStatus.timeout: - return RunCodeResult( - errorMessage: kTimeoutErrorText, - graph: prevGraph, - log: prevLog, - output: kTimeoutErrorText, - pipelineUuid: pipelineUuid, - sdk: prevResult.sdk, - status: status, - ); - - case RunCodeStatus.runError: - final output = await _client.getRunErrorOutput(pipelineUuid); - return RunCodeResult( - graph: prevGraph, - log: prevLog, - output: output.output, - pipelineUuid: pipelineUuid, - sdk: prevResult.sdk, - status: status, - ); - - case RunCodeStatus.validationError: - final output = await _client.getValidationErrorOutput(pipelineUuid); - return RunCodeResult( - graph: prevGraph, - log: prevLog, - output: output.output, - sdk: prevResult.sdk, - status: status, - ); - - case RunCodeStatus.preparationError: - final output = await _client.getPreparationErrorOutput(pipelineUuid); - return RunCodeResult( - graph: prevGraph, - log: prevLog, - output: output.output, - sdk: prevResult.sdk, - status: status, - ); - - case RunCodeStatus.unknownError: - return RunCodeResult( - errorMessage: kUnknownErrorText, - graph: prevGraph, - log: prevLog, - output: kUnknownErrorText, - pipelineUuid: pipelineUuid, - sdk: prevResult.sdk, - status: status, - ); - - case RunCodeStatus.executing: - final responses = await Future.wait([ - _client.getRunOutput(pipelineUuid), - _client.getLogOutput(pipelineUuid), - prevGraph.isEmpty - ? _client.getGraphOutput(pipelineUuid) - : Future.value(OutputResponse(output: prevGraph)), - ]); - final output = responses[0]; - final log = responses[1]; - final graph = responses[2]; - return RunCodeResult( - graph: graph.output, - log: prevLog + log.output, - output: prevOutput + output.output, - pipelineUuid: pipelineUuid, - sdk: prevResult.sdk, - status: status, - ); - - case RunCodeStatus.finished: - final responses = await Future.wait([ - _client.getRunOutput(pipelineUuid), - _client.getLogOutput(pipelineUuid), - _client.getRunErrorOutput(pipelineUuid), - prevGraph.isEmpty - ? _client.getGraphOutput(pipelineUuid) - : Future.value(OutputResponse(output: prevGraph)), - ]); - final output = responses[0]; - final log = responses[1]; - final error = responses[2]; - final graph = responses[3]; - return RunCodeResult( - graph: graph.output, - log: prevLog + log.output, - output: prevOutput + output.output + error.output, - pipelineUuid: pipelineUuid, - sdk: prevResult.sdk, - status: status, - ); - - default: - return RunCodeResult( - graph: prevGraph, - log: prevLog, - pipelineUuid: pipelineUuid, - sdk: prevResult.sdk, - status: status, - ); - } - } -} diff --git a/playground/frontend/playground_components/lib/src/repositories/models/run_code_error.dart b/playground/frontend/playground_components/lib/src/repositories/models/run_code_error.dart index 68379a2473bb2..6b5eb34013621 100644 --- a/playground/frontend/playground_components/lib/src/repositories/models/run_code_error.dart +++ b/playground/frontend/playground_components/lib/src/repositories/models/run_code_error.dart @@ -23,3 +23,9 @@ class RunCodeError implements Exception { this.message, }); } + +class RunCodeResourceExhaustedError extends RunCodeError { + const RunCodeResourceExhaustedError({ + required super.message, + }); +} diff --git a/playground/frontend/playground_components/lib/src/repositories/models/run_code_result.dart b/playground/frontend/playground_components/lib/src/repositories/models/run_code_result.dart index 506fe54f633a8..7ca6e104f1d38 100644 --- a/playground/frontend/playground_components/lib/src/repositories/models/run_code_result.dart +++ b/playground/frontend/playground_components/lib/src/repositories/models/run_code_result.dart @@ -21,28 +21,30 @@ import 'package:equatable/equatable.dart'; import '../../models/sdk.dart'; enum RunCodeStatus { - unspecified, - preparation, - preparationError, - validationError, - compiling, + cancelled, compileError, + compiling, executing, - runError, finished, + preparation, + preparationError, + runError, timeout, unknownError, + unspecified, + validationError, } -const kFinishedStatuses = [ - RunCodeStatus.unknownError, - RunCodeStatus.timeout, +const kFinishedStatuses = { + RunCodeStatus.cancelled, RunCodeStatus.compileError, + RunCodeStatus.finished, + RunCodeStatus.preparationError, RunCodeStatus.runError, + RunCodeStatus.timeout, + RunCodeStatus.unknownError, RunCodeStatus.validationError, - RunCodeStatus.preparationError, - RunCodeStatus.finished, -]; +}; class RunCodeResult with EquatableMixin { final String? errorMessage; diff --git a/playground/frontend/playground_components/lib/src/widgets/run_or_cancel_button.dart b/playground/frontend/playground_components/lib/src/widgets/run_or_cancel_button.dart index 444a33e743635..79c00dabdf565 100644 --- a/playground/frontend/playground_components/lib/src/widgets/run_or_cancel_button.dart +++ b/playground/frontend/playground_components/lib/src/widgets/run_or_cancel_button.dart @@ -24,6 +24,7 @@ import '../controllers/playground_controller.dart'; import '../models/toast.dart'; import '../models/toast_type.dart'; import '../playground_components.dart'; +import '../repositories/models/run_code_result.dart'; import 'run_button.dart'; class RunOrCancelButton extends StatelessWidget { @@ -49,11 +50,13 @@ class RunOrCancelButton extends StatelessWidget { (_) => PlaygroundComponents.toastNotifier.add(_getErrorToast()), ); }, - runCode: () { + runCode: () async { beforeRun?.call(); - playgroundController.codeRunner.runCode( - onFinish: () => onComplete?.call(playgroundController.codeRunner), - ); + final runner = playgroundController.codeRunner; + await runner.runCode(); + if (runner.result?.status == RunCodeStatus.finished) { + onComplete?.call(playgroundController.codeRunner); + } }, ); } diff --git a/playground/frontend/playground_components/test/src/controllers/code_runner_test.dart b/playground/frontend/playground_components/test/src/controllers/code_runner_test.dart new file mode 100644 index 0000000000000..ff3766c43f39a --- /dev/null +++ b/playground/frontend/playground_components/test/src/controllers/code_runner_test.dart @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import 'package:flutter_test/flutter_test.dart'; +import 'package:mockito/annotations.dart'; +import 'package:mockito/mockito.dart'; +import 'package:playground_components/src/cache/example_cache.dart'; +import 'package:playground_components/src/controllers/code_runner.dart'; +import 'package:playground_components/src/controllers/example_loaders/examples_loader.dart'; +import 'package:playground_components/src/controllers/snippet_editing_controller.dart'; +import 'package:playground_components/src/models/sdk.dart'; +import 'package:playground_components/src/repositories/code_client/code_client.dart'; +import 'package:playground_components/src/repositories/models/check_status_response.dart'; +import 'package:playground_components/src/repositories/models/output_response.dart'; +import 'package:playground_components/src/repositories/models/run_code_response.dart'; +import 'package:playground_components/src/repositories/models/run_code_result.dart'; + +import 'code_runner_test.mocks.dart'; + +const _sdk = Sdk.java; + +const kPipelineUuid = '1234'; +const kRunOutput = 'RunOutput'; +const kLogOutput = 'LogOutput'; +const kCompileOutput = 'CompileOutput'; +const kGraphOutput = 'GraphOutput'; +const kRunErrorOutput = 'RunErrorOutput'; +const kPreparationErrorOutput = 'PreparationErrorOutput'; +const kValidationErrorOutput = 'ValidationErrorOutput'; + +const kRunCodeResponse = RunCodeResponse(pipelineUuid: kPipelineUuid); +const kFinishedStatusResponse = CheckStatusResponse( + status: RunCodeStatus.finished, +); +const kRunErrorStatusResponse = CheckStatusResponse( + status: RunCodeStatus.runError, +); +const kCompileErrorStatusResponse = CheckStatusResponse( + status: RunCodeStatus.compileError, +); +const kValidationErrorStatusResponse = CheckStatusResponse( + status: RunCodeStatus.validationError, +); +const kPreparationErrorStatusResponse = CheckStatusResponse( + status: RunCodeStatus.preparationError, +); + +const kRunOutputResponse = OutputResponse(output: kRunOutput); +const kLogOutputResponse = OutputResponse(output: kLogOutput); +const kCompileOutputResponse = OutputResponse(output: kCompileOutput); +const kRunErrorOutputResponse = OutputResponse(output: kRunErrorOutput); +const kGraphResponse = OutputResponse(output: kGraphOutput); + +const kValidationErrorOutputResponse = + OutputResponse(output: kValidationErrorOutput); + +const kPreparationErrorOutputResponse = + OutputResponse(output: kPreparationErrorOutput); + +@GenerateMocks([CodeClient, ExamplesLoader, ExampleCache]) +void main() { + var results = []; + var client = MockCodeClient(); + var runner = CodeRunner( + snippetEditingControllerGetter: () => SnippetEditingController(sdk: _sdk), + ); + + setUp(() async { + results = []; + client = MockCodeClient(); + runner = CodeRunner( + snippetEditingControllerGetter: () => SnippetEditingController(sdk: _sdk), + codeClient: client, + ); + + runner.addListener(() { + results.add(runner.result); + }); + + when(client.runCode(any)).thenAnswer( + (_) async => kRunCodeResponse, + ); + when(client.getCompileOutput(kPipelineUuid)).thenAnswer( + (_) async => kCompileOutputResponse, + ); + when(client.getRunOutput(kPipelineUuid)).thenAnswer( + (_) async => kRunOutputResponse, + ); + when(client.getRunErrorOutput(kPipelineUuid)).thenAnswer( + (_) async => kRunErrorOutputResponse, + ); + when(client.getLogOutput(kPipelineUuid)).thenAnswer( + (_) async => kLogOutputResponse, + ); + when(client.getGraphOutput(kPipelineUuid)).thenAnswer( + (_) async => kGraphResponse, + ); + when(client.getValidationErrorOutput(kPipelineUuid)).thenAnswer( + (_) async => kValidationErrorOutputResponse, + ); + when(client.getPreparationErrorOutput(kPipelineUuid)).thenAnswer( + (_) async => kPreparationErrorOutputResponse, + ); + }); + + group('CodeRunner.runCode', () { + test('finished', () async { + when(client.checkStatus(kPipelineUuid)).thenAnswer( + (_) async => kFinishedStatusResponse, + ); + + await runner.runCode(); + + expect( + results, + const [ + RunCodeResult( + log: kProcessingStartedText, + sdk: _sdk, + status: RunCodeStatus.preparation, + ), + RunCodeResult( + pipelineUuid: kPipelineUuid, + output: kRunOutput + kRunErrorOutput, + log: kProcessingStartedText + kLogOutput, + graph: kGraphOutput, + sdk: _sdk, + status: RunCodeStatus.finished, + ), + ], + ); + + // compile output should not be called + verifyNever(client.getCompileOutput(kPipelineUuid)); + }); + + test('compileError', () async { + when(client.checkStatus(kPipelineUuid)).thenAnswer( + (_) async => kCompileErrorStatusResponse, + ); + + await runner.runCode(); + + expect( + results, + const [ + RunCodeResult( + log: kProcessingStartedText, + sdk: _sdk, + status: RunCodeStatus.preparation, + ), + RunCodeResult( + graph: '', + log: kProcessingStartedText, + output: kCompileOutput, + pipelineUuid: kPipelineUuid, + sdk: _sdk, + status: RunCodeStatus.compileError, + ), + ], + ); + }); + + test('validationError', () async { + when(client.checkStatus(kPipelineUuid)).thenAnswer( + (_) async => kValidationErrorStatusResponse, + ); + + await runner.runCode(); + + expect( + results, + const [ + RunCodeResult( + log: kProcessingStartedText, + sdk: _sdk, + status: RunCodeStatus.preparation, + ), + RunCodeResult( + graph: '', + log: kProcessingStartedText, + output: kValidationErrorOutput, + sdk: _sdk, + status: RunCodeStatus.validationError, + ), + ], + ); + }); + + test('preparationError', () async { + when(client.checkStatus(kPipelineUuid)).thenAnswer( + (_) async => kPreparationErrorStatusResponse, + ); + + await runner.runCode(); + + expect( + results, + const [ + RunCodeResult( + log: kProcessingStartedText, + sdk: _sdk, + status: RunCodeStatus.preparation, + ), + RunCodeResult( + graph: '', + log: kProcessingStartedText, + output: kPreparationErrorOutput, + sdk: _sdk, + status: RunCodeStatus.preparationError, + ), + ], + ); + }); + + test('runError', () async { + when(client.checkStatus(kPipelineUuid)).thenAnswer( + (_) async => kRunErrorStatusResponse, + ); + + await runner.runCode(); + + expect( + results, + const [ + RunCodeResult( + log: kProcessingStartedText, + sdk: _sdk, + status: RunCodeStatus.preparation, + ), + RunCodeResult( + graph: '', + log: kProcessingStartedText, + output: kRunErrorOutput, + pipelineUuid: kPipelineUuid, + sdk: _sdk, + status: RunCodeStatus.runError, + ), + ], + ); + }); + }); +} diff --git a/playground/frontend/playground_components/test/src/controllers/code_runner_test.mocks.dart b/playground/frontend/playground_components/test/src/controllers/code_runner_test.mocks.dart new file mode 100644 index 0000000000000..892ad2d46dfb1 --- /dev/null +++ b/playground/frontend/playground_components/test/src/controllers/code_runner_test.mocks.dart @@ -0,0 +1,564 @@ +// Mocks generated by Mockito 5.3.2 from annotations +// in playground_components/test/src/controllers/code_runner_test.dart. +// Do not manually edit this file. + +// ignore_for_file: no_leading_underscores_for_library_prefixes +import 'dart:async' as _i9; +import 'dart:ui' as _i21; + +import 'package:mockito/mockito.dart' as _i1; +import 'package:playground_components/src/api/v1/api.pb.dart' as _i2; +import 'package:playground_components/src/cache/example_cache.dart' as _i15; +import 'package:playground_components/src/controllers/example_loaders/example_loader_factory.dart' + as _i6; +import 'package:playground_components/src/controllers/example_loaders/examples_loader.dart' + as _i12; +import 'package:playground_components/src/controllers/playground_controller.dart' + as _i13; +import 'package:playground_components/src/models/category_with_examples.dart' + as _i16; +import 'package:playground_components/src/models/example.dart' as _i7; +import 'package:playground_components/src/models/example_base.dart' as _i20; +import 'package:playground_components/src/models/example_loading_descriptors/examples_loading_descriptor.dart' + as _i14; +import 'package:playground_components/src/models/example_view_options.dart' + as _i18; +import 'package:playground_components/src/models/loading_status.dart' as _i17; +import 'package:playground_components/src/models/sdk.dart' as _i10; +import 'package:playground_components/src/models/snippet_file.dart' as _i19; +import 'package:playground_components/src/repositories/code_client/code_client.dart' + as _i8; +import 'package:playground_components/src/repositories/models/check_status_response.dart' + as _i4; +import 'package:playground_components/src/repositories/models/output_response.dart' + as _i5; +import 'package:playground_components/src/repositories/models/run_code_request.dart' + as _i11; +import 'package:playground_components/src/repositories/models/run_code_response.dart' + as _i3; + +// ignore_for_file: type=lint +// ignore_for_file: avoid_redundant_argument_values +// ignore_for_file: avoid_setters_without_getters +// ignore_for_file: comment_references +// ignore_for_file: implementation_imports +// ignore_for_file: invalid_use_of_visible_for_testing_member +// ignore_for_file: prefer_const_constructors +// ignore_for_file: unnecessary_parenthesis +// ignore_for_file: camel_case_types +// ignore_for_file: subtype_of_sealed_class + +class _FakeGetMetadataResponse_0 extends _i1.SmartFake + implements _i2.GetMetadataResponse { + _FakeGetMetadataResponse_0( + Object parent, + Invocation parentInvocation, + ) : super( + parent, + parentInvocation, + ); +} + +class _FakeRunCodeResponse_1 extends _i1.SmartFake + implements _i3.RunCodeResponse { + _FakeRunCodeResponse_1( + Object parent, + Invocation parentInvocation, + ) : super( + parent, + parentInvocation, + ); +} + +class _FakeCheckStatusResponse_2 extends _i1.SmartFake + implements _i4.CheckStatusResponse { + _FakeCheckStatusResponse_2( + Object parent, + Invocation parentInvocation, + ) : super( + parent, + parentInvocation, + ); +} + +class _FakeOutputResponse_3 extends _i1.SmartFake + implements _i5.OutputResponse { + _FakeOutputResponse_3( + Object parent, + Invocation parentInvocation, + ) : super( + parent, + parentInvocation, + ); +} + +class _FakeExampleLoaderFactory_4 extends _i1.SmartFake + implements _i6.ExampleLoaderFactory { + _FakeExampleLoaderFactory_4( + Object parent, + Invocation parentInvocation, + ) : super( + parent, + parentInvocation, + ); +} + +class _FakeExample_5 extends _i1.SmartFake implements _i7.Example { + _FakeExample_5( + Object parent, + Invocation parentInvocation, + ) : super( + parent, + parentInvocation, + ); +} + +/// A class which mocks [CodeClient]. +/// +/// See the documentation for Mockito's code generation for more information. +class MockCodeClient extends _i1.Mock implements _i8.CodeClient { + MockCodeClient() { + _i1.throwOnMissingStub(this); + } + + @override + _i9.Future<_i2.GetMetadataResponse> getMetadata(_i10.Sdk? sdk) => + (super.noSuchMethod( + Invocation.method( + #getMetadata, + [sdk], + ), + returnValue: _i9.Future<_i2.GetMetadataResponse>.value( + _FakeGetMetadataResponse_0( + this, + Invocation.method( + #getMetadata, + [sdk], + ), + )), + ) as _i9.Future<_i2.GetMetadataResponse>); + @override + _i9.Future<_i3.RunCodeResponse> runCode(_i11.RunCodeRequest? request) => + (super.noSuchMethod( + Invocation.method( + #runCode, + [request], + ), + returnValue: + _i9.Future<_i3.RunCodeResponse>.value(_FakeRunCodeResponse_1( + this, + Invocation.method( + #runCode, + [request], + ), + )), + ) as _i9.Future<_i3.RunCodeResponse>); + @override + _i9.Future cancelExecution(String? pipelineUuid) => (super.noSuchMethod( + Invocation.method( + #cancelExecution, + [pipelineUuid], + ), + returnValue: _i9.Future.value(), + returnValueForMissingStub: _i9.Future.value(), + ) as _i9.Future); + @override + _i9.Future<_i4.CheckStatusResponse> checkStatus(String? pipelineUuid) => + (super.noSuchMethod( + Invocation.method( + #checkStatus, + [pipelineUuid], + ), + returnValue: _i9.Future<_i4.CheckStatusResponse>.value( + _FakeCheckStatusResponse_2( + this, + Invocation.method( + #checkStatus, + [pipelineUuid], + ), + )), + ) as _i9.Future<_i4.CheckStatusResponse>); + @override + _i9.Future<_i5.OutputResponse> getCompileOutput(String? pipelineUuid) => + (super.noSuchMethod( + Invocation.method( + #getCompileOutput, + [pipelineUuid], + ), + returnValue: _i9.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( + this, + Invocation.method( + #getCompileOutput, + [pipelineUuid], + ), + )), + ) as _i9.Future<_i5.OutputResponse>); + @override + _i9.Future<_i5.OutputResponse> getRunOutput(String? pipelineUuid) => + (super.noSuchMethod( + Invocation.method( + #getRunOutput, + [pipelineUuid], + ), + returnValue: _i9.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( + this, + Invocation.method( + #getRunOutput, + [pipelineUuid], + ), + )), + ) as _i9.Future<_i5.OutputResponse>); + @override + _i9.Future<_i5.OutputResponse> getLogOutput(String? pipelineUuid) => + (super.noSuchMethod( + Invocation.method( + #getLogOutput, + [pipelineUuid], + ), + returnValue: _i9.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( + this, + Invocation.method( + #getLogOutput, + [pipelineUuid], + ), + )), + ) as _i9.Future<_i5.OutputResponse>); + @override + _i9.Future<_i5.OutputResponse> getRunErrorOutput(String? pipelineUuid) => + (super.noSuchMethod( + Invocation.method( + #getRunErrorOutput, + [pipelineUuid], + ), + returnValue: _i9.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( + this, + Invocation.method( + #getRunErrorOutput, + [pipelineUuid], + ), + )), + ) as _i9.Future<_i5.OutputResponse>); + @override + _i9.Future<_i5.OutputResponse> getValidationErrorOutput( + String? pipelineUuid) => + (super.noSuchMethod( + Invocation.method( + #getValidationErrorOutput, + [pipelineUuid], + ), + returnValue: _i9.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( + this, + Invocation.method( + #getValidationErrorOutput, + [pipelineUuid], + ), + )), + ) as _i9.Future<_i5.OutputResponse>); + @override + _i9.Future<_i5.OutputResponse> getPreparationErrorOutput( + String? pipelineUuid) => + (super.noSuchMethod( + Invocation.method( + #getPreparationErrorOutput, + [pipelineUuid], + ), + returnValue: _i9.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( + this, + Invocation.method( + #getPreparationErrorOutput, + [pipelineUuid], + ), + )), + ) as _i9.Future<_i5.OutputResponse>); + @override + _i9.Future<_i5.OutputResponse> getGraphOutput(String? pipelineUuid) => + (super.noSuchMethod( + Invocation.method( + #getGraphOutput, + [pipelineUuid], + ), + returnValue: _i9.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( + this, + Invocation.method( + #getGraphOutput, + [pipelineUuid], + ), + )), + ) as _i9.Future<_i5.OutputResponse>); +} + +/// A class which mocks [ExamplesLoader]. +/// +/// See the documentation for Mockito's code generation for more information. +class MockExamplesLoader extends _i1.Mock implements _i12.ExamplesLoader { + MockExamplesLoader() { + _i1.throwOnMissingStub(this); + } + + @override + _i6.ExampleLoaderFactory get defaultFactory => (super.noSuchMethod( + Invocation.getter(#defaultFactory), + returnValue: _FakeExampleLoaderFactory_4( + this, + Invocation.getter(#defaultFactory), + ), + ) as _i6.ExampleLoaderFactory); + @override + void setPlaygroundController(_i13.PlaygroundController? value) => + super.noSuchMethod( + Invocation.method( + #setPlaygroundController, + [value], + ), + returnValueForMissingStub: null, + ); + @override + _i9.Future loadIfNew(_i14.ExamplesLoadingDescriptor? descriptor) => + (super.noSuchMethod( + Invocation.method( + #loadIfNew, + [descriptor], + ), + returnValue: _i9.Future.value(), + returnValueForMissingStub: _i9.Future.value(), + ) as _i9.Future); + @override + _i9.Future load(_i14.ExamplesLoadingDescriptor? descriptor) => + (super.noSuchMethod( + Invocation.method( + #load, + [descriptor], + ), + returnValue: _i9.Future.value(), + returnValueForMissingStub: _i9.Future.value(), + ) as _i9.Future); + @override + _i9.Future loadDefaultIfAny(_i10.Sdk? sdk) => (super.noSuchMethod( + Invocation.method( + #loadDefaultIfAny, + [sdk], + ), + returnValue: _i9.Future.value(), + returnValueForMissingStub: _i9.Future.value(), + ) as _i9.Future); +} + +/// A class which mocks [ExampleCache]. +/// +/// See the documentation for Mockito's code generation for more information. +class MockExampleCache extends _i1.Mock implements _i15.ExampleCache { + MockExampleCache() { + _i1.throwOnMissingStub(this); + } + + @override + Map<_i10.Sdk, List<_i16.CategoryWithExamples>> get categoryListsBySdk => + (super.noSuchMethod( + Invocation.getter(#categoryListsBySdk), + returnValue: <_i10.Sdk, List<_i16.CategoryWithExamples>>{}, + ) as Map<_i10.Sdk, List<_i16.CategoryWithExamples>>); + @override + Map<_i10.Sdk, _i7.Example> get defaultExamplesBySdk => (super.noSuchMethod( + Invocation.getter(#defaultExamplesBySdk), + returnValue: <_i10.Sdk, _i7.Example>{}, + ) as Map<_i10.Sdk, _i7.Example>); + @override + bool get isSelectorOpened => (super.noSuchMethod( + Invocation.getter(#isSelectorOpened), + returnValue: false, + ) as bool); + @override + set isSelectorOpened(bool? _isSelectorOpened) => super.noSuchMethod( + Invocation.setter( + #isSelectorOpened, + _isSelectorOpened, + ), + returnValueForMissingStub: null, + ); + @override + _i9.Future get allExamplesFuture => (super.noSuchMethod( + Invocation.getter(#allExamplesFuture), + returnValue: _i9.Future.value(), + ) as _i9.Future); + @override + _i17.LoadingStatus get catalogStatus => (super.noSuchMethod( + Invocation.getter(#catalogStatus), + returnValue: _i17.LoadingStatus.loading, + ) as _i17.LoadingStatus); + @override + bool get hasListeners => (super.noSuchMethod( + Invocation.getter(#hasListeners), + returnValue: false, + ) as bool); + @override + _i9.Future loadAllPrecompiledObjectsIfNot() => (super.noSuchMethod( + Invocation.method( + #loadAllPrecompiledObjectsIfNot, + [], + ), + returnValue: _i9.Future.value(), + returnValueForMissingStub: _i9.Future.value(), + ) as _i9.Future); + @override + List<_i16.CategoryWithExamples> getCategories(_i10.Sdk? sdk) => + (super.noSuchMethod( + Invocation.method( + #getCategories, + [sdk], + ), + returnValue: <_i16.CategoryWithExamples>[], + ) as List<_i16.CategoryWithExamples>); + @override + _i9.Future<_i7.Example> getPrecompiledObject( + String? path, + _i10.Sdk? sdk, + ) => + (super.noSuchMethod( + Invocation.method( + #getPrecompiledObject, + [ + path, + sdk, + ], + ), + returnValue: _i9.Future<_i7.Example>.value(_FakeExample_5( + this, + Invocation.method( + #getPrecompiledObject, + [ + path, + sdk, + ], + ), + )), + ) as _i9.Future<_i7.Example>); + @override + _i9.Future<_i7.Example> loadSharedExample( + String? id, { + required _i18.ExampleViewOptions? viewOptions, + }) => + (super.noSuchMethod( + Invocation.method( + #loadSharedExample, + [id], + {#viewOptions: viewOptions}, + ), + returnValue: _i9.Future<_i7.Example>.value(_FakeExample_5( + this, + Invocation.method( + #loadSharedExample, + [id], + {#viewOptions: viewOptions}, + ), + )), + ) as _i9.Future<_i7.Example>); + @override + _i9.Future saveSnippet({ + required List<_i19.SnippetFile>? files, + required _i10.Sdk? sdk, + required String? pipelineOptions, + }) => + (super.noSuchMethod( + Invocation.method( + #saveSnippet, + [], + { + #files: files, + #sdk: sdk, + #pipelineOptions: pipelineOptions, + }, + ), + returnValue: _i9.Future.value(''), + ) as _i9.Future); + @override + _i9.Future<_i7.Example> loadExampleInfo(_i20.ExampleBase? example) => + (super.noSuchMethod( + Invocation.method( + #loadExampleInfo, + [example], + ), + returnValue: _i9.Future<_i7.Example>.value(_FakeExample_5( + this, + Invocation.method( + #loadExampleInfo, + [example], + ), + )), + ) as _i9.Future<_i7.Example>); + @override + void setSelectorOpened(bool? value) => super.noSuchMethod( + Invocation.method( + #setSelectorOpened, + [value], + ), + returnValueForMissingStub: null, + ); + @override + _i9.Future<_i7.Example?> getDefaultExampleBySdk(_i10.Sdk? sdk) => + (super.noSuchMethod( + Invocation.method( + #getDefaultExampleBySdk, + [sdk], + ), + returnValue: _i9.Future<_i7.Example?>.value(), + ) as _i9.Future<_i7.Example?>); + @override + _i9.Future loadDefaultPrecompiledObjects() => (super.noSuchMethod( + Invocation.method( + #loadDefaultPrecompiledObjects, + [], + ), + returnValue: _i9.Future.value(), + returnValueForMissingStub: _i9.Future.value(), + ) as _i9.Future); + @override + _i9.Future loadDefaultPrecompiledObjectsIfNot() => (super.noSuchMethod( + Invocation.method( + #loadDefaultPrecompiledObjectsIfNot, + [], + ), + returnValue: _i9.Future.value(), + returnValueForMissingStub: _i9.Future.value(), + ) as _i9.Future); + @override + _i9.Future<_i20.ExampleBase?> getCatalogExampleByPath(String? path) => + (super.noSuchMethod( + Invocation.method( + #getCatalogExampleByPath, + [path], + ), + returnValue: _i9.Future<_i20.ExampleBase?>.value(), + ) as _i9.Future<_i20.ExampleBase?>); + @override + void addListener(_i21.VoidCallback? listener) => super.noSuchMethod( + Invocation.method( + #addListener, + [listener], + ), + returnValueForMissingStub: null, + ); + @override + void removeListener(_i21.VoidCallback? listener) => super.noSuchMethod( + Invocation.method( + #removeListener, + [listener], + ), + returnValueForMissingStub: null, + ); + @override + void dispose() => super.noSuchMethod( + Invocation.method( + #dispose, + [], + ), + returnValueForMissingStub: null, + ); + @override + void notifyListeners() => super.noSuchMethod( + Invocation.method( + #notifyListeners, + [], + ), + returnValueForMissingStub: null, + ); +} diff --git a/playground/frontend/playground_components/test/src/repositories/code_repository_test.dart b/playground/frontend/playground_components/test/src/repositories/code_repository_test.dart deleted file mode 100644 index 2b07ea9d2ebbf..0000000000000 --- a/playground/frontend/playground_components/test/src/repositories/code_repository_test.dart +++ /dev/null @@ -1,389 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import 'package:flutter_test/flutter_test.dart'; -import 'package:mockito/annotations.dart'; -import 'package:mockito/mockito.dart'; -import 'package:playground_components/src/models/sdk.dart'; -import 'package:playground_components/src/models/snippet_file.dart'; -import 'package:playground_components/src/repositories/code_client/code_client.dart'; -import 'package:playground_components/src/repositories/code_repository.dart'; -import 'package:playground_components/src/repositories/models/check_status_response.dart'; -import 'package:playground_components/src/repositories/models/output_response.dart'; -import 'package:playground_components/src/repositories/models/run_code_request.dart'; -import 'package:playground_components/src/repositories/models/run_code_response.dart'; -import 'package:playground_components/src/repositories/models/run_code_result.dart'; - -import 'code_repository_test.mocks.dart'; - -const kRequestMock = RunCodeRequest( - files: [SnippetFile(content: 'code', isMain: true)], - sdk: Sdk.java, - pipelineOptions: {}, - datasets: [], -); - -const kPipelineUuid = '1234'; -const kRunOutput = 'RunOutput'; -const kLogOutput = 'LogOutput'; -const kCompileOutput = 'CompileOutput'; -const kGraphOutput = 'GraphOutput'; -const kRunErrorOutput = 'RunErrorOutput'; -const kPreparationErrorOutput = 'PreparationErrorOutput'; -const kValidationErrorOutput = 'ValidationErrorOutput'; - -const kRunCodeResponse = RunCodeResponse(pipelineUuid: kPipelineUuid); -const kFinishedStatusResponse = CheckStatusResponse( - status: RunCodeStatus.finished, -); -const kErrorStatusResponse = CheckStatusResponse( - status: RunCodeStatus.unknownError, -); -const kRunErrorStatusResponse = CheckStatusResponse( - status: RunCodeStatus.runError, -); -const kExecutingStatusResponse = CheckStatusResponse( - status: RunCodeStatus.executing, -); -const kCompileErrorStatusResponse = CheckStatusResponse( - status: RunCodeStatus.compileError, -); -const kValidationErrorStatusResponse = CheckStatusResponse( - status: RunCodeStatus.validationError, -); -const kPreparationErrorStatusResponse = CheckStatusResponse( - status: RunCodeStatus.preparationError, -); - -const kRunOutputResponse = OutputResponse(output: kRunOutput); -const kLogOutputResponse = OutputResponse(output: kLogOutput); -const kCompileOutputResponse = OutputResponse(output: kCompileOutput); -const kRunErrorOutputResponse = OutputResponse(output: kRunErrorOutput); -const kGraphResponse = OutputResponse(output: kGraphOutput); - -const kValidationErrorOutputResponse = - OutputResponse(output: kValidationErrorOutput); - -const kPreparationErrorOutputResponse = - OutputResponse(output: kPreparationErrorOutput); - -@GenerateMocks([CodeClient]) -void main() { - group('CodeRepository runCode', () { - test('should complete finished run code request', () async { - // stubs - final client = MockCodeClient(); - when(client.runCode(kRequestMock)).thenAnswer( - (_) async => kRunCodeResponse, - ); - when(client.checkStatus(kPipelineUuid)).thenAnswer( - (_) async => kFinishedStatusResponse, - ); - when(client.getRunOutput(kPipelineUuid)).thenAnswer( - (_) async => kRunOutputResponse, - ); - when(client.getCompileOutput(kPipelineUuid)).thenAnswer( - (_) async => kCompileOutputResponse, - ); - when(client.getRunErrorOutput(kPipelineUuid)).thenAnswer( - (_) async => kRunErrorOutputResponse, - ); - when(client.getLogOutput(kPipelineUuid)).thenAnswer( - (_) async => kLogOutputResponse, - ); - when(client.getGraphOutput(kPipelineUuid)).thenAnswer( - (_) async => kGraphResponse, - ); - - // test variables - final repository = CodeRepository(client: client); - final stream = repository.runCode(kRequestMock); - - // test assertion - await expectLater( - stream, - emitsInOrder([ - RunCodeResult( - log: kProcessingStartedText, - sdk: kRequestMock.sdk, - status: RunCodeStatus.preparation, - ), - RunCodeResult( - graph: kGraphOutput, - log: kProcessingStartedText + kLogOutput, - output: kRunOutput + kRunErrorOutput, - pipelineUuid: kPipelineUuid, - sdk: kRequestMock.sdk, - status: RunCodeStatus.finished, - ), - ]), - ); - // compile output should not be called - verifyNever(client.getCompileOutput(kPipelineUuid)); - }); - - test('should return output from compilation if failed', () async { - // stubs - final client = MockCodeClient(); - when(client.runCode(kRequestMock)).thenAnswer( - (_) async => kRunCodeResponse, - ); - when(client.checkStatus(kPipelineUuid)).thenAnswer( - (_) async => kCompileErrorStatusResponse, - ); - when(client.getCompileOutput(kPipelineUuid)).thenAnswer( - (_) async => kCompileOutputResponse, - ); - when(client.getRunOutput(kPipelineUuid)).thenAnswer( - (_) async => kRunOutputResponse, - ); - when(client.getLogOutput(kPipelineUuid)).thenAnswer( - (_) async => kLogOutputResponse, - ); - when(client.getGraphOutput(kPipelineUuid)).thenAnswer( - (_) async => kGraphResponse, - ); - - // test variables - final repository = CodeRepository(client: client); - final stream = repository.runCode(kRequestMock); - - // test assertion - await expectLater( - stream, - emitsInOrder([ - RunCodeResult( - log: kProcessingStartedText, - sdk: kRequestMock.sdk, - status: RunCodeStatus.preparation, - ), - RunCodeResult( - graph: '', - log: kProcessingStartedText, - output: kCompileOutput, - pipelineUuid: kPipelineUuid, - sdk: kRequestMock.sdk, - status: RunCodeStatus.compileError, - ), - ]), - ); - }); - - test('should return validation error output for validation error', - () async { - // stubs - final client = MockCodeClient(); - when(client.runCode(kRequestMock)).thenAnswer( - (_) async => kRunCodeResponse, - ); - when(client.checkStatus(kPipelineUuid)).thenAnswer( - (_) async => kValidationErrorStatusResponse, - ); - when(client.getValidationErrorOutput(kPipelineUuid)).thenAnswer( - (_) async => kValidationErrorOutputResponse, - ); - when(client.getGraphOutput(kPipelineUuid)).thenAnswer( - (_) async => kGraphResponse, - ); - - // test variables - final repository = CodeRepository(client: client); - final stream = repository.runCode(kRequestMock); - - // test assertion - await expectLater( - stream, - emitsInOrder([ - RunCodeResult( - log: kProcessingStartedText, - sdk: kRequestMock.sdk, - status: RunCodeStatus.preparation, - ), - RunCodeResult( - graph: '', - log: kProcessingStartedText, - output: kValidationErrorOutput, - sdk: kRequestMock.sdk, - status: RunCodeStatus.validationError, - ), - ]), - ); - }); - - test('should return preparation error output for preparation error', - () async { - // stubs - final client = MockCodeClient(); - when(client.runCode(kRequestMock)).thenAnswer( - (_) async => kRunCodeResponse, - ); - when(client.checkStatus(kPipelineUuid)).thenAnswer( - (_) async => kPreparationErrorStatusResponse, - ); - when(client.getPreparationErrorOutput(kPipelineUuid)).thenAnswer( - (_) async => kPreparationErrorOutputResponse, - ); - when(client.getGraphOutput(kPipelineUuid)).thenAnswer( - (_) async => kGraphResponse, - ); - - // test variables - final repository = CodeRepository(client: client); - final stream = repository.runCode(kRequestMock); - - // test assertion - await expectLater( - stream, - emitsInOrder([ - RunCodeResult( - sdk: kRequestMock.sdk, - log: kProcessingStartedText, - status: RunCodeStatus.preparation, - ), - RunCodeResult( - graph: '', - log: kProcessingStartedText, - output: kPreparationErrorOutput, - sdk: kRequestMock.sdk, - status: RunCodeStatus.preparationError, - ), - ]), - ); - }); - - test('should return output from runError if failed while running', - () async { - // stubs - final client = MockCodeClient(); - when(client.runCode(kRequestMock)).thenAnswer( - (_) async => kRunCodeResponse, - ); - when(client.checkStatus(kPipelineUuid)).thenAnswer( - (_) async => kRunErrorStatusResponse, - ); - when(client.getCompileOutput(kPipelineUuid)).thenAnswer( - (_) async => kCompileOutputResponse, - ); - when(client.getRunOutput(kPipelineUuid)).thenAnswer( - (_) async => kRunOutputResponse, - ); - when(client.getRunErrorOutput(kPipelineUuid)).thenAnswer( - (_) async => kRunErrorOutputResponse, - ); - when(client.getLogOutput(kPipelineUuid)).thenAnswer( - (_) async => kLogOutputResponse, - ); - when(client.getGraphOutput(kPipelineUuid)).thenAnswer( - (_) async => kGraphResponse, - ); - - // test variables - final repository = CodeRepository(client: client); - final stream = repository.runCode(kRequestMock); - - // test assertion - await expectLater( - stream, - emitsInOrder([ - RunCodeResult( - log: kProcessingStartedText, - sdk: kRequestMock.sdk, - status: RunCodeStatus.preparation, - ), - RunCodeResult( - graph: '', - log: kProcessingStartedText, - output: kRunErrorOutput, - pipelineUuid: kPipelineUuid, - sdk: kRequestMock.sdk, - status: RunCodeStatus.runError, - ), - ]), - ); - }); - }); - - test('should return full output and log using streaming api when finished', - () async { - // stubs - final client = MockCodeClient(); - when(client.runCode(kRequestMock)).thenAnswer( - (_) async => kRunCodeResponse, - ); - final answers = [ - kExecutingStatusResponse, - kExecutingStatusResponse, - kFinishedStatusResponse - ]; - when(client.checkStatus(kPipelineUuid)).thenAnswer( - (_) async => answers.removeAt(0), - ); - when(client.getRunOutput(kPipelineUuid)).thenAnswer( - (_) async => kRunOutputResponse, - ); - when(client.getRunErrorOutput(kPipelineUuid)).thenAnswer( - (_) async => kRunErrorOutputResponse, - ); - when(client.getLogOutput(kPipelineUuid)).thenAnswer( - (_) async => kLogOutputResponse, - ); - when(client.getGraphOutput(kPipelineUuid)).thenAnswer( - (_) async => kGraphResponse, - ); - - // test variables - final repository = CodeRepository(client: client); - final stream = repository.runCode(kRequestMock); - - // test assertion - await expectLater( - stream, - emitsInOrder([ - RunCodeResult( - log: kProcessingStartedText, - sdk: kRequestMock.sdk, - status: RunCodeStatus.preparation, - ), - RunCodeResult( - graph: kGraphOutput, - log: kProcessingStartedText + kLogOutput, - output: kRunOutput, - pipelineUuid: kPipelineUuid, - sdk: kRequestMock.sdk, - status: RunCodeStatus.executing, - ), - RunCodeResult( - graph: kGraphOutput, - log: kProcessingStartedText + kLogOutput * 2, - output: kRunOutput * 2, - pipelineUuid: kPipelineUuid, - sdk: kRequestMock.sdk, - status: RunCodeStatus.executing, - ), - RunCodeResult( - graph: kGraphOutput, - log: kProcessingStartedText + kLogOutput * 3, - output: kRunOutput * 3 + kRunErrorOutput, - pipelineUuid: kPipelineUuid, - sdk: kRequestMock.sdk, - status: RunCodeStatus.finished, - ), - ]), - ); - }); -} diff --git a/playground/frontend/playground_components/test/src/repositories/code_repository_test.mocks.dart b/playground/frontend/playground_components/test/src/repositories/code_repository_test.mocks.dart deleted file mode 100644 index c109bae2a1577..0000000000000 --- a/playground/frontend/playground_components/test/src/repositories/code_repository_test.mocks.dart +++ /dev/null @@ -1,249 +0,0 @@ -// Mocks generated by Mockito 5.3.2 from annotations -// in playground_components/test/src/repositories/code_repository_test.dart. -// Do not manually edit this file. - -// ignore_for_file: no_leading_underscores_for_library_prefixes -import 'dart:async' as _i7; - -import 'package:mockito/mockito.dart' as _i1; -import 'package:playground_components/src/api/v1/api.pb.dart' as _i2; -import 'package:playground_components/src/models/sdk.dart' as _i8; -import 'package:playground_components/src/repositories/code_client/code_client.dart' - as _i6; -import 'package:playground_components/src/repositories/models/check_status_response.dart' - as _i4; -import 'package:playground_components/src/repositories/models/output_response.dart' - as _i5; -import 'package:playground_components/src/repositories/models/run_code_request.dart' - as _i9; -import 'package:playground_components/src/repositories/models/run_code_response.dart' - as _i3; - -// ignore_for_file: type=lint -// ignore_for_file: avoid_redundant_argument_values -// ignore_for_file: avoid_setters_without_getters -// ignore_for_file: comment_references -// ignore_for_file: implementation_imports -// ignore_for_file: invalid_use_of_visible_for_testing_member -// ignore_for_file: prefer_const_constructors -// ignore_for_file: unnecessary_parenthesis -// ignore_for_file: camel_case_types -// ignore_for_file: subtype_of_sealed_class - -class _FakeGetMetadataResponse_0 extends _i1.SmartFake - implements _i2.GetMetadataResponse { - _FakeGetMetadataResponse_0( - Object parent, - Invocation parentInvocation, - ) : super( - parent, - parentInvocation, - ); -} - -class _FakeRunCodeResponse_1 extends _i1.SmartFake - implements _i3.RunCodeResponse { - _FakeRunCodeResponse_1( - Object parent, - Invocation parentInvocation, - ) : super( - parent, - parentInvocation, - ); -} - -class _FakeCheckStatusResponse_2 extends _i1.SmartFake - implements _i4.CheckStatusResponse { - _FakeCheckStatusResponse_2( - Object parent, - Invocation parentInvocation, - ) : super( - parent, - parentInvocation, - ); -} - -class _FakeOutputResponse_3 extends _i1.SmartFake - implements _i5.OutputResponse { - _FakeOutputResponse_3( - Object parent, - Invocation parentInvocation, - ) : super( - parent, - parentInvocation, - ); -} - -/// A class which mocks [CodeClient]. -/// -/// See the documentation for Mockito's code generation for more information. -class MockCodeClient extends _i1.Mock implements _i6.CodeClient { - MockCodeClient() { - _i1.throwOnMissingStub(this); - } - - @override - _i7.Future<_i2.GetMetadataResponse> getMetadata(_i8.Sdk? sdk) => - (super.noSuchMethod( - Invocation.method( - #getMetadata, - [sdk], - ), - returnValue: _i7.Future<_i2.GetMetadataResponse>.value( - _FakeGetMetadataResponse_0( - this, - Invocation.method( - #getMetadata, - [sdk], - ), - )), - ) as _i7.Future<_i2.GetMetadataResponse>); - @override - _i7.Future<_i3.RunCodeResponse> runCode(_i9.RunCodeRequest? request) => - (super.noSuchMethod( - Invocation.method( - #runCode, - [request], - ), - returnValue: - _i7.Future<_i3.RunCodeResponse>.value(_FakeRunCodeResponse_1( - this, - Invocation.method( - #runCode, - [request], - ), - )), - ) as _i7.Future<_i3.RunCodeResponse>); - @override - _i7.Future cancelExecution(String? pipelineUuid) => (super.noSuchMethod( - Invocation.method( - #cancelExecution, - [pipelineUuid], - ), - returnValue: _i7.Future.value(), - returnValueForMissingStub: _i7.Future.value(), - ) as _i7.Future); - @override - _i7.Future<_i4.CheckStatusResponse> checkStatus(String? pipelineUuid) => - (super.noSuchMethod( - Invocation.method( - #checkStatus, - [pipelineUuid], - ), - returnValue: _i7.Future<_i4.CheckStatusResponse>.value( - _FakeCheckStatusResponse_2( - this, - Invocation.method( - #checkStatus, - [pipelineUuid], - ), - )), - ) as _i7.Future<_i4.CheckStatusResponse>); - @override - _i7.Future<_i5.OutputResponse> getCompileOutput(String? pipelineUuid) => - (super.noSuchMethod( - Invocation.method( - #getCompileOutput, - [pipelineUuid], - ), - returnValue: _i7.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( - this, - Invocation.method( - #getCompileOutput, - [pipelineUuid], - ), - )), - ) as _i7.Future<_i5.OutputResponse>); - @override - _i7.Future<_i5.OutputResponse> getRunOutput(String? pipelineUuid) => - (super.noSuchMethod( - Invocation.method( - #getRunOutput, - [pipelineUuid], - ), - returnValue: _i7.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( - this, - Invocation.method( - #getRunOutput, - [pipelineUuid], - ), - )), - ) as _i7.Future<_i5.OutputResponse>); - @override - _i7.Future<_i5.OutputResponse> getLogOutput(String? pipelineUuid) => - (super.noSuchMethod( - Invocation.method( - #getLogOutput, - [pipelineUuid], - ), - returnValue: _i7.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( - this, - Invocation.method( - #getLogOutput, - [pipelineUuid], - ), - )), - ) as _i7.Future<_i5.OutputResponse>); - @override - _i7.Future<_i5.OutputResponse> getRunErrorOutput(String? pipelineUuid) => - (super.noSuchMethod( - Invocation.method( - #getRunErrorOutput, - [pipelineUuid], - ), - returnValue: _i7.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( - this, - Invocation.method( - #getRunErrorOutput, - [pipelineUuid], - ), - )), - ) as _i7.Future<_i5.OutputResponse>); - @override - _i7.Future<_i5.OutputResponse> getValidationErrorOutput( - String? pipelineUuid) => - (super.noSuchMethod( - Invocation.method( - #getValidationErrorOutput, - [pipelineUuid], - ), - returnValue: _i7.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( - this, - Invocation.method( - #getValidationErrorOutput, - [pipelineUuid], - ), - )), - ) as _i7.Future<_i5.OutputResponse>); - @override - _i7.Future<_i5.OutputResponse> getPreparationErrorOutput( - String? pipelineUuid) => - (super.noSuchMethod( - Invocation.method( - #getPreparationErrorOutput, - [pipelineUuid], - ), - returnValue: _i7.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( - this, - Invocation.method( - #getPreparationErrorOutput, - [pipelineUuid], - ), - )), - ) as _i7.Future<_i5.OutputResponse>); - @override - _i7.Future<_i5.OutputResponse> getGraphOutput(String? pipelineUuid) => - (super.noSuchMethod( - Invocation.method( - #getGraphOutput, - [pipelineUuid], - ), - returnValue: _i7.Future<_i5.OutputResponse>.value(_FakeOutputResponse_3( - this, - Invocation.method( - #getGraphOutput, - [pipelineUuid], - ), - )), - ) as _i7.Future<_i5.OutputResponse>); -} diff --git a/playground/frontend/playground_components_dev/lib/src/widget_tester.dart b/playground/frontend/playground_components_dev/lib/src/widget_tester.dart index 83c7f416ba8c6..507f3bdaaae1a 100644 --- a/playground/frontend/playground_components_dev/lib/src/widget_tester.dart +++ b/playground/frontend/playground_components_dev/lib/src/widget_tester.dart @@ -138,7 +138,7 @@ extension WidgetTesterExtension on WidgetTester { expect(codeRunner.isCodeRunning, false); expect( PlaygroundComponents.analyticsService.lastEvent, - isA(), // Cached finish does not fire events. + isA(), ); await pumpAndSettle(); // Let the UI catch up. @@ -147,7 +147,6 @@ extension WidgetTesterExtension on WidgetTester { expectOutput(example, this); } - /// Runs and expects that the execution is as fast as it should be for cache. Future modifyRunExpectReal(ExampleDescriptor example) async { modifyCodeController(); @@ -162,6 +161,9 @@ extension WidgetTesterExtension on WidgetTester { expect(actualText, isNot(startsWith(kCachedResultsLog))); expectOutput(example, this); + // Animation stops just before the analytics event is fired, wait a bit. + await Future.delayed(const Duration(seconds: 1)); + final event = PlaygroundComponents.analyticsService.lastEvent; expect(event, isA());