From 9aaca71994f89a91a881f91232bacad1f3637b70 Mon Sep 17 00:00:00 2001 From: Shengquan Ni <13672781+shengquan-ni@users.noreply.github.com> Date: Thu, 31 Oct 2024 17:29:19 -0700 Subject: [PATCH 1/2] renaming --- .../architecture/python/proxy_message.proto | 37 + ...lcommands.proto => control_commands.proto} | 110 +- ...rolreturns.proto => control_returns.proto} | 43 +- ...service.proto => controller_service.proto} | 5 +- ...sterservice.proto => tester_service.proto} | 5 +- ...rkerservice.proto => worker_service.proto} | 5 +- .../sendsemantics/partitionings.proto | 36 +- .../architecture/worker/statistics.proto | 2 +- ...actormessage.proto => actor_message.proto} | 9 +- .../amber/engine/common/ambermessage.proto | 31 - ...lidentity.proto => virtual_identity.proto} | 11 +- .../ics/amber/engine/common/workflow.proto | 17 +- .../engine/common/workflow_metrics.proto | 41 + ...ate.proto => workflow_runtime_state.proto} | 41 +- .../amber/engine/architecture/rpc/__init__.py | 835 +++--- .../uci/ics/amber/engine/common/__init__.py | 113 +- .../proto/edu/uci/ics/texera/web/__init__.py | 2 +- .../amber/clustering/ClusterListener.scala | 13 +- .../common/AkkaActorRefMappingService.scala | 4 +- .../common/AkkaActorService.scala | 2 +- .../common/AkkaMessageTransferService.scala | 2 +- .../architecture/common/AmberProcessor.scala | 6 +- .../common/ProcessingStepCursor.scala | 2 +- .../architecture/common/WorkflowActor.scala | 2 +- .../architecture/controller/ClientEvent.scala | 4 +- .../architecture/controller/Controller.scala | 6 +- ...ControllerAsyncRPCHandlerInitializer.scala | 6 +- .../controller/ControllerProcessor.scala | 2 +- .../controller/ControllerTimerService.scala | 6 +- .../controller/GlobalReplayManager.scala | 2 +- .../controller/execution/ExecutionUtils.scala | 5 +- .../controller/execution/LinkExecution.scala | 2 +- .../execution/OperatorExecution.scala | 7 +- .../execution/RegionExecution.scala | 7 +- .../execution/WorkflowExecution.scala | 6 +- .../ChannelMarkerHandler.scala | 8 +- .../ConsoleMessageHandler.scala | 4 +- .../promisehandlers/DebugCommandHandler.scala | 6 +- .../EvaluatePythonExpressionHandler.scala | 6 +- .../promisehandlers/LinkWorkersHandler.scala | 8 +- .../promisehandlers/PauseHandler.scala | 6 +- .../PortCompletedHandler.scala | 6 +- .../QueryWorkerStatisticsHandler.scala | 4 +- .../promisehandlers/ResumeHandler.scala | 4 +- .../RetrieveWorkflowStateHandler.scala | 12 +- .../RetryWorkflowHandler.scala | 6 +- .../StartWorkflowHandler.scala | 6 +- .../TakeGlobalCheckpointHandler.scala | 12 +- .../WorkerExecutionCompletedHandler.scala | 6 +- .../WorkerStateUpdatedHandler.scala | 4 +- .../layer/WorkerExecution.scala | 6 +- .../logreplay/EmptyReplayLogger.scala | 2 +- .../logreplay/OrderEnforcer.scala | 2 +- .../logreplay/ReplayLogGenerator.scala | 2 +- .../logreplay/ReplayLogManager.scala | 2 +- .../architecture/logreplay/ReplayLogger.scala | 2 +- .../logreplay/ReplayLoggerImpl.scala | 2 +- .../logreplay/ReplayOrderEnforcer.scala | 2 +- .../messaginglayer/AmberFIFOChannel.scala | 4 +- .../messaginglayer/InputGateway.scala | 2 +- .../messaginglayer/InputManager.scala | 4 +- .../messaginglayer/NetworkInputGateway.scala | 2 +- .../messaginglayer/NetworkOutputGateway.scala | 4 +- .../messaginglayer/OutputManager.scala | 6 +- .../messaginglayer/WorkerPort.scala | 2 +- .../messaginglayer/WorkerTimerService.scala | 6 +- .../pythonworker/PythonProxyClient.scala | 33 +- .../pythonworker/PythonProxyServer.scala | 9 +- .../pythonworker/PythonWorkflowWorker.scala | 4 +- .../WorkerBatchInternalQueue.scala | 6 +- .../CostBasedRegionPlanGenerator.scala | 4 +- .../ExpansionGreedyRegionPlanGenerator.scala | 4 +- .../architecture/scheduling/Region.scala | 4 +- .../RegionExecutionCoordinator.scala | 28 +- .../architecture/scheduling/RegionPlan.scala | 2 +- .../scheduling/RegionPlanGenerator.scala | 4 +- .../WorkflowExecutionCoordinator.scala | 2 +- .../scheduling/config/ChannelConfig.scala | 4 +- .../scheduling/config/LinkConfig.scala | 4 +- .../scheduling/config/ResourceConfig.scala | 4 +- .../scheduling/config/WorkerConfig.scala | 2 +- .../resourcePolicies/ResourceAllocator.scala | 4 +- .../partitioners/BroadcastPartitioner.scala | 4 +- .../HashBasedShufflePartitioner.scala | 4 +- .../partitioners/OneToOnePartitioner.scala | 4 +- .../partitioners/Partitioner.scala | 2 +- .../RangeBasedShufflePartitioner.scala | 4 +- .../partitioners/RoundRobinPartitioner.scala | 4 +- .../worker/ChannelMarkerManager.scala | 6 +- .../engine/architecture/worker/DPThread.scala | 10 +- .../architecture/worker/DataProcessor.scala | 14 +- .../DataProcessorRPCHandlerInitializer.scala | 8 +- .../architecture/worker/PauseManager.scala | 2 +- .../architecture/worker/PauseType.scala | 2 +- .../architecture/worker/WorkflowWorker.scala | 6 +- .../managers/SerializationManager.scala | 6 +- .../worker/managers/StatisticsManager.scala | 4 +- .../AddInputChannelHandler.scala | 6 +- .../AddPartitioningHandler.scala | 6 +- .../promisehandlers/AssignPortHandler.scala | 4 +- .../FinalizeCheckpointHandler.scala | 4 +- .../FlushNetworkBufferHandler.scala | 6 +- .../InitializeExecutorHandler.scala | 4 +- .../promisehandlers/OpenExecutorHandler.scala | 4 +- .../worker/promisehandlers/PauseHandler.scala | 6 +- .../PrepareCheckpointHandler.scala | 6 +- .../QueryStatisticsHandler.scala | 6 +- .../promisehandlers/ResumeHandler.scala | 6 +- .../RetrieveStateHandler.scala | 4 +- .../worker/promisehandlers/StartHandler.scala | 12 +- .../amber/engine/common/AmberLogging.scala | 2 +- .../engine/common/CheckpointSupport.scala | 2 +- .../uci/ics/amber/engine/common/Utils.scala | 1 - .../engine/common/VirtualIdentityUtils.scala | 7 - .../WorkflowRuntimeException.scala | 2 +- .../common/ambermessage/RecoveryPayload.scala | 2 +- .../common/ambermessage/WorkflowMessage.scala | 2 +- .../engine/common/client/AmberClient.scala | 8 +- .../engine/common/client/ClientActor.scala | 12 +- .../common/executor/OperatorExecutor.scala | 2 +- .../executor/SinkOperatorExecutor.scala | 2 +- .../executor/SourceOperatorExecutor.scala | 2 +- .../engine/common/model/PhysicalOp.scala | 3 +- .../engine/common/model/PhysicalPlan.scala | 4 +- .../engine/common/model/WorkflowContext.scala | 2 +- .../engine/common/model/tuple/TupleLike.scala | 2 +- .../engine/common/rpc/AsyncRPCClient.scala | 18 +- .../rpc/AsyncRPCHandlerInitializer.scala | 7 +- .../engine/common/rpc/AsyncRPCServer.scala | 6 +- .../common/statetransition/StateManager.scala | 2 +- .../statetransition/WorkerStateManager.scala | 6 +- .../storage/SequentialRecordStorage.scala | 2 +- .../common/{virtualidentity => }/util.scala | 2 +- .../edu/uci/ics/amber/error/ErrorUtils.scala | 14 +- .../ics/texera/web/TexeraWebApplication.scala | 37 +- .../texera/web/WorkflowLifecycleManager.scala | 6 +- .../websocket/event/WorkflowErrorEvent.scala | 2 +- .../event/python/ConsoleUpdateEvent.scala | 2 +- .../PythonExpressionEvaluateResponse.scala | 2 +- .../resource/SchemaPropagationResource.scala | 2 +- .../WorkflowCompilationResource.scala | 2 +- .../resource/WorkflowWebsocketResource.scala | 17 +- .../workflow/WorkflowExecutionsResource.scala | 2 +- .../service/EmailNotificationService.scala | 3 +- .../web/service/ExecutionConsoleService.scala | 8 +- .../web/service/ExecutionResultService.scala | 25 +- .../web/service/ExecutionRuntimeService.scala | 21 +- .../web/service/ExecutionStatsService.scala | 39 +- .../ExecutionsMetadataPersistService.scala | 2 +- .../FriesReconfigurationAlgorithm.scala | 4 +- .../web/service/ResultExportService.scala | 2 +- .../web/service/WorkflowCacheChecker.scala | 2 +- .../web/service/WorkflowEmailNotifier.scala | 4 +- .../service/WorkflowExecutionService.scala | 14 +- .../texera/web/service/WorkflowService.scala | 22 +- .../ExecutionReconfigurationStore.scala | 2 +- .../web/storage/ExecutionStateStore.scala | 8 +- .../web/storage/WorkflowResultStore.scala | 2 +- .../metadata/OperatorMetadataGenerator.scala | 2 +- .../workflow/common/operators/LogicalOp.scala | 4 +- .../operators/PythonOperatorDescriptor.scala | 2 +- .../operators/filter/FilterOpDesc.scala | 2 +- .../common/operators/map/MapOpDesc.scala | 2 +- .../common/storage/OpResultStorage.scala | 2 +- .../common/workflow/LogicalLink.scala | 4 +- .../common/workflow/LogicalPlan.scala | 4 +- .../common/workflow/LogicalPort.scala | 2 +- .../workflow/SinkInjectionTransformer.scala | 4 +- .../common/workflow/WorkflowCompiler.scala | 8 +- .../operators/aggregate/AggregateOpDesc.scala | 4 +- .../CartesianProductOpDesc.scala | 4 +- .../dictionary/DictionaryMatcherOpDesc.scala | 4 +- .../difference/DifferenceOpDesc.scala | 4 +- .../operators/distinct/DistinctOpDesc.scala | 4 +- .../download/BulkDownloaderOpDesc.scala | 4 +- .../operators/dummy/DummyOpDesc.scala | 2 +- .../filter/SpecializedFilterOpDesc.java | 10 +- .../operators/hashJoin/HashJoinOpDesc.scala | 4 +- ...gingFaceIrisLogisticRegressionOpDesc.scala | 2 +- .../HuggingFaceSentimentAnalysisOpDesc.scala | 2 +- .../HuggingFaceSpamSMSDetectionOpDesc.scala | 2 +- .../HuggingFaceTextSummarizationOpDesc.scala | 2 +- .../operators/intersect/IntersectOpDesc.scala | 4 +- .../intervalJoin/IntervalJoinOpDesc.scala | 4 +- .../keywordSearch/KeywordSearchOpDesc.scala | 4 +- .../operators/limit/LimitOpDesc.scala | 4 +- .../Scorer/MachineLearningScorerOpDesc.scala | 2 +- .../base/SklearnAdvancedBaseDesc.scala | 2 +- .../projection/ProjectionOpDesc.scala | 4 +- .../RandomKSamplingOpDesc.scala | 4 +- .../operators/regex/RegexOpDesc.scala | 4 +- .../ReservoirSamplingOpDesc.scala | 4 +- .../sentiment/SentimentAnalysisOpDesc.scala | 4 +- .../sink/managed/ProgressiveSinkOpDesc.java | 12 +- .../operators/sklearn/SklearnMLOpDesc.scala | 2 +- .../sklearn/SklearnPredictionOpDesc.scala | 2 +- .../workflow/operators/sort/SortOpDesc.scala | 2 +- .../sortPartitions/SortPartitionsOpDesc.scala | 4 +- .../reddit/RedditSearchSourceOpDesc.scala | 2 +- .../apis/twitter/TwitterSourceOpDesc.scala | 2 +- ...TwitterFullArchiveSearchSourceOpDesc.scala | 2 +- .../v2/TwitterSearchSourceOpDesc.scala | 2 +- .../source/cache/CacheSourceOpDesc.scala | 4 +- .../source/fetcher/URLFetcherOpDesc.scala | 4 +- .../source/scan/FileScanSourceOpDesc.scala | 2 +- .../source/scan/ScanSourceOpDesc.scala | 2 +- .../source/scan/csv/CSVScanSourceOpDesc.scala | 2 +- .../source/scan/csv/CSVScanSourceOpExec.scala | 2 +- .../csv/ParallelCSVScanSourceOpDesc.scala | 2 +- .../scan/csvOld/CSVOldScanSourceOpDesc.scala | 2 +- .../scan/json/JSONLScanSourceOpDesc.scala | 2 +- .../scan/text/TextInputSourceOpDesc.scala | 4 +- .../sql/asterixdb/AsterixDBSourceOpDesc.scala | 4 +- .../source/sql/mysql/MySQLSourceOpDesc.scala | 4 +- .../postgresql/PostgreSQLSourceOpDesc.scala | 4 +- .../operators/split/SplitOpDesc.scala | 4 +- .../operators/split/SplitOpExec.scala | 2 +- .../SymmetricDifferenceOpDesc.scala | 4 +- .../typecasting/TypeCastingOpDesc.java | 10 +- .../operators/udf/java/JavaUDFOpDesc.scala | 4 +- .../DualInputPortsPythonUDFOpDescV2.scala | 8 +- .../python/PythonLambdaFunctionOpDesc.scala | 2 +- .../udf/python/PythonTableReducerOpDesc.scala | 2 +- .../udf/python/PythonUDFOpDescV2.scala | 4 +- .../source/PythonUDFSourceOpDescV2.java | 10 +- .../workflow/operators/udf/r/RUDFOpDesc.scala | 4 +- .../operators/udf/r/RUDFSourceOpDesc.java | 10 +- .../operators/union/UnionOpDesc.scala | 4 +- .../unneststring/UnnestStringOpDesc.scala | 4 +- .../visualization/DotPlot/DotPlotOpDesc.scala | 2 +- .../IcicleChart/IcicleChartOpDesc.scala | 2 +- .../ImageViz/ImageVisualizerOpDesc.scala | 2 +- .../ScatterMatrixChartOpDesc.scala | 2 +- .../barChart/BarChartOpDesc.scala | 2 +- .../visualization/boxPlot/BoxPlotOpDesc.scala | 2 +- .../bubbleChart/BubbleChartOpDesc.scala | 2 +- .../CandlestickChartOpDesc.scala | 2 +- .../ContinuousErrorBandsOpDesc.scala | 2 +- .../contourPlot/ContourPlotOpDesc.scala | 2 +- .../dumbbellPlot/DumbbellPlotOpDesc.scala | 2 +- .../FigureFactoryTableOpDesc.scala | 2 +- .../filledAreaPlot/FilledAreaPlotOpDesc.scala | 2 +- .../funnelPlot/FunnelPlotOpDesc.scala | 2 +- .../ganttChart/GanttChartOpDesc.scala | 2 +- .../visualization/heatMap/HeatMapOpDesc.scala | 2 +- .../hierarchychart/HierarchyChartOpDesc.scala | 2 +- .../histogram/HistogramChartOpDesc.scala | 2 +- .../visualization/htmlviz/HtmlVizOpDesc.scala | 4 +- .../lineChart/LineChartOpDesc.scala | 2 +- .../pieChart/PieChartOpDesc.scala | 2 +- .../quiverPlot/QuiverPlotOpDesc.scala | 2 +- .../sankeyDiagram/SankeyDiagramOpDesc.scala | 2 +- .../scatter3DChart/Scatter3dChartOpDesc.scala | 2 +- .../scatterplot/ScatterplotOpDesc.scala | 2 +- .../tablesChart/TablesPlotOpDesc.scala | 2 +- .../ternaryPlot/TernaryPlotOpDesc.scala | 2 +- .../visualization/urlviz/UrlVizOpDesc.scala | 4 +- .../waterfallChart/WaterfallOpDesc.scala | 2 +- .../wordCloud/WordCloudOpDesc.scala | 2 +- .../python}/ControlPayloadV2.scala | 88 +- .../python/ProxyMessageProto.scala | 58 + .../python}/PythonActorMessage.scala | 62 +- .../python}/PythonControlMessage.scala | 74 +- .../python}/PythonDataHeader.scala | 52 +- .../AsyncRPCContext.scala | 70 +- .../ChannelMarkerPayload.scala | 170 +- .../ChannelMarkerType.scala | 10 +- .../ConsoleMessage.scala | 44 +- .../ConsoleMessageType.scala | 10 +- .../rpc/ControlCommandsProto.scala | 228 ++ .../ControlInvocation.scala | 84 +- .../ControlRequest.scala | 2252 ++++++++--------- .../{controlreturns => }/ControlReturn.scala | 942 +++---- .../rpc/ControlReturnsProto.scala | 111 + .../rpc/ControllerServiceFs2Grpc.scala | 124 + .../ControllerServiceGrpc.scala | 310 +-- .../rpc/ControllerServiceProto.scala | 67 + .../{controlreturns => }/ErrorLanguage.scala | 10 +- .../{controlreturns => }/EvaluatedValue.scala | 66 +- .../architecture/rpc/RPCTesterFs2Grpc.scala | 89 + .../{testerservice => }/RPCTesterGrpc.scala | 210 +- .../ReturnInvocation.scala | 58 +- .../architecture/rpc/TesterServiceProto.scala | 54 + .../rpc/{controlreturns => }/TypedValue.scala | 26 +- .../rpc/WorkerServiceFs2Grpc.scala | 138 + .../WorkerServiceGrpc.scala | 350 +-- .../architecture/rpc/WorkerServiceProto.scala | 68 + .../WorkflowReconfigureRequest.scala | 50 +- .../ControlcommandsProto.scala | 227 -- .../ControllerServiceFs2Grpc.scala | 124 - .../ControllerserviceProto.scala | 67 - .../controlreturns/ControlreturnsProto.scala | 112 - .../rpc/testerservice/RPCTesterFs2Grpc.scala | 89 - .../testerservice/TesterserviceProto.scala | 54 - .../workerservice/WorkerServiceFs2Grpc.scala | 138 - .../workerservice/WorkerserviceProto.scala | 68 - .../{partitionings => }/Partitioning.scala | 424 ++-- .../sendsemantics/PartitioningsProto.scala | 68 + .../partitionings/PartitioningsProto.scala | 68 - .../PortTupleCountMapping.scala | 44 +- .../{statistics => }/StatisticsProto.scala | 14 +- .../{statistics => }/WorkerMetrics.scala | 62 +- .../worker/{statistics => }/WorkerState.scala | 6 +- .../{statistics => }/WorkerStatistics.scala | 62 +- .../{actormessage => }/ActorCommand.scala | 174 +- ...ageProto.scala => ActorMessageProto.scala} | 26 +- .../ActorVirtualIdentity.scala | 26 +- .../BreakpointFault.scala | 72 +- .../ChannelIdentity.scala | 76 +- .../ChannelMarkerIdentity.scala | 26 +- .../EvaluatedValueList.scala | 46 +- .../ExecutionBreakpointStore.scala | 106 +- .../ExecutionConsoleStore.scala | 106 +- .../ExecutionIdentity.scala | 26 +- .../ExecutionMetadataStore.scala | 88 +- .../ExecutionStatsStore.scala | 186 +- .../FatalErrorType.scala | 10 +- .../common/{workflow => }/InputPort.scala | 68 +- .../OperatorBreakpoints.scala | 46 +- .../OperatorConsole.scala | 126 +- .../OperatorIdentity.scala | 26 +- .../OperatorMetrics.scala | 66 +- .../OperatorStatistics.scala | 66 +- .../OperatorWorkerMapping.scala | 30 +- .../common/{workflow => }/OutputPort.scala | 46 +- .../common/{workflow => }/PhysicalLink.scala | 118 +- .../PhysicalOpIdentity.scala | 52 +- .../common/{workflow => }/PortIdentity.scala | 22 +- .../engine/common/VirtualIdentityProto.scala | 49 + .../WorkflowAggregatedState.scala | 10 +- .../WorkflowFatalError.scala | 48 +- .../WorkflowIdentity.scala | 26 +- .../engine/common/WorkflowMetricsProto.scala | 54 + .../amber/engine/common/WorkflowProto.scala | 52 + .../common/WorkflowRuntimeStateProto.scala | 98 + .../ambermessage/AmbermessageProto.scala | 53 - .../VirtualidentityProto.scala | 49 - .../common/workflow/WorkflowProto.scala | 52 - .../WorkflowruntimestateProto.scala | 111 - .../control/TrivialControlSpec.scala | 10 +- .../control/utils/ChainHandler.scala | 3 +- .../control/utils/CollectHandler.scala | 3 +- .../control/utils/ErrorHandler.scala | 3 +- .../control/utils/MultiCallHandler.scala | 5 +- .../control/utils/NestedHandler.scala | 3 +- .../control/utils/PingPongHandler.scala | 3 +- .../control/utils/RecursionHandler.scala | 3 +- .../TesterAsyncRPCHandlerInitializer.scala | 6 +- .../control/utils/TrivialControlTester.scala | 6 +- .../NetworkInputGatewaySpec.scala | 2 +- .../messaginglayer/OutputManagerSpec.scala | 6 +- .../RangeBasedShuffleSpec.scala | 4 +- .../PythonWorkflowWorkerSpec.scala | 4 +- ...pansionGreedyRegionPlanGeneratorSpec.scala | 4 +- .../architecture/worker/DPThreadSpec.scala | 10 +- .../worker/DataProcessorSpec.scala | 12 +- .../architecture/worker/WorkerSpec.scala | 12 +- .../engine/e2e/BatchSizePropagationSpec.scala | 4 +- .../amber/engine/e2e/DataProcessingSpec.scala | 8 +- .../uci/ics/amber/engine/e2e/PauseSpec.scala | 6 +- .../faulttolerance/CheckpointSpec.scala | 31 +- .../engine/faulttolerance/LoggingSpec.scala | 14 +- .../engine/faulttolerance/ReplaySpec.scala | 8 +- .../workflow/SchemaPropagationSpec.scala | 4 +- .../intersect/IntersectOpExecSpec.scala | 4 +- .../intervalJoin/IntervalOpExecSpec.scala | 4 +- .../scan/csv/CSVScanSourceOpDescSpec.scala | 2 +- .../unneststring/UnnestStringOpExecSpec.scala | 2 +- 368 files changed, 6158 insertions(+), 6283 deletions(-) create mode 100644 core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/python/proxy_message.proto rename core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/{controlcommands.proto => control_commands.proto} (60%) rename core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/{controlreturns.proto => control_returns.proto} (66%) rename core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/{controllerservice.proto => controller_service.proto} (89%) rename core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/{testerservice.proto => tester_service.proto} (83%) rename core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/{workerservice.proto => worker_service.proto} (89%) rename core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/{actormessage.proto => actor_message.proto} (69%) delete mode 100644 core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/ambermessage.proto rename core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/{virtualidentity.proto => virtual_identity.proto} (65%) create mode 100644 core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflow_metrics.proto rename core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/{workflowruntimestate.proto => workflow_runtime_state.proto} (57%) rename core/amber/src/main/scala/edu/uci/ics/amber/engine/common/{virtualidentity => }/util.scala (89%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/{common/ambermessage => architecture/python}/ControlPayloadV2.scala (60%) create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/ProxyMessageProto.scala rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/{common/actormessage => architecture/python}/PythonActorMessage.scala (56%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/{common/ambermessage => architecture/python}/PythonControlMessage.scala (58%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/{common/ambermessage => architecture/python}/PythonDataHeader.scala (68%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlcommands => }/AsyncRPCContext.scala (58%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlcommands => }/ChannelMarkerPayload.scala (57%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlcommands => }/ChannelMarkerType.scala (78%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlcommands => }/ConsoleMessage.scala (80%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlcommands => }/ConsoleMessageType.scala (81%) create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlCommandsProto.scala rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlcommands => }/ControlInvocation.scala (60%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlcommands => }/ControlRequest.scala (59%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlreturns => }/ControlReturn.scala (57%) create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlReturnsProto.scala create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControllerServiceFs2Grpc.scala rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controllerservice => }/ControllerServiceGrpc.scala (56%) create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControllerServiceProto.scala rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlreturns => }/ErrorLanguage.scala (77%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlreturns => }/EvaluatedValue.scala (66%) create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/RPCTesterFs2Grpc.scala rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{testerservice => }/RPCTesterGrpc.scala (59%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlreturns => }/ReturnInvocation.scala (60%) create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/TesterServiceProto.scala rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlreturns => }/TypedValue.scala (87%) create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkerServiceFs2Grpc.scala rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{workerservice => }/WorkerServiceGrpc.scala (56%) create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkerServiceProto.scala rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/{controlcommands => }/WorkflowReconfigureRequest.scala (69%) delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ControlcommandsProto.scala delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controllerservice/ControllerServiceFs2Grpc.scala delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controllerservice/ControllerserviceProto.scala delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ControlreturnsProto.scala delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/testerservice/RPCTesterFs2Grpc.scala delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/testerservice/TesterserviceProto.scala delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/workerservice/WorkerServiceFs2Grpc.scala delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/workerservice/WorkerserviceProto.scala rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/{partitionings => }/Partitioning.scala (64%) create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/PartitioningsProto.scala delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/partitionings/PartitioningsProto.scala rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/{statistics => }/PortTupleCountMapping.scala (71%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/{statistics => }/StatisticsProto.scala (87%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/{statistics => }/WorkerMetrics.scala (67%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/{statistics => }/WorkerState.scala (87%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/{statistics => }/WorkerStatistics.scala (75%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{actormessage => }/ActorCommand.scala (55%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{actormessage/ActormessageProto.scala => ActorMessageProto.scala} (53%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{virtualidentity => }/ActorVirtualIdentity.scala (75%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/BreakpointFault.scala (74%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{virtualidentity => }/ChannelIdentity.scala (62%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{virtualidentity => }/ChannelMarkerIdentity.scala (75%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/EvaluatedValueList.scala (66%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/ExecutionBreakpointStore.scala (57%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/ExecutionConsoleStore.scala (57%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{virtualidentity => }/ExecutionIdentity.scala (75%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/ExecutionMetadataStore.scala (58%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/ExecutionStatsStore.scala (55%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/FatalErrorType.scala (78%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflow => }/InputPort.scala (72%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/OperatorBreakpoints.scala (64%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/OperatorConsole.scala (58%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{virtualidentity => }/OperatorIdentity.scala (76%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/OperatorMetrics.scala (54%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/OperatorStatistics.scala (75%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/OperatorWorkerMapping.scala (79%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflow => }/OutputPort.scala (75%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflow => }/PhysicalLink.scala (54%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{virtualidentity => }/PhysicalOpIdentity.scala (67%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflow => }/PortIdentity.scala (83%) create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/VirtualIdentityProto.scala rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/{architecture/rpc/controlreturns => common}/WorkflowAggregatedState.scala (88%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{workflowruntimestate => }/WorkflowFatalError.scala (78%) rename core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/{virtualidentity => }/WorkflowIdentity.scala (75%) create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowMetricsProto.scala create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowProto.scala create mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowRuntimeStateProto.scala delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/AmbermessageProto.scala delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/VirtualidentityProto.scala delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/WorkflowProto.scala delete mode 100644 core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/WorkflowruntimestateProto.scala diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/python/proxy_message.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/python/proxy_message.proto new file mode 100644 index 00000000000..b2025416c60 --- /dev/null +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/python/proxy_message.proto @@ -0,0 +1,37 @@ +syntax = "proto3"; + +package edu.uci.ics.amber.engine.architecture.python; + +import "edu/uci/ics/amber/engine/architecture/rpc/control_commands.proto"; +import "edu/uci/ics/amber/engine/architecture/rpc/control_returns.proto"; +import "edu/uci/ics/amber/engine/common/virtual_identity.proto"; +import "edu/uci/ics/amber/engine/common/actor_message.proto"; +import "scalapb/scalapb.proto"; + +option (scalapb.options) = { + scope: FILE, + preserve_unknown_fields: false + no_default_values_in_constructor: true + flat_package: true +}; + +message ControlPayloadV2 { + oneof value { + architecture.rpc.ControlInvocation control_invocation = 1; + architecture.rpc.ReturnInvocation return_invocation = 2; + } +} + +message PythonDataHeader { + common.ActorVirtualIdentity tag = 1 [(scalapb.field).no_box = true]; + string payload_type = 2; +} + +message PythonControlMessage { + common.ActorVirtualIdentity tag = 1 [(scalapb.field).no_box = true]; + ControlPayloadV2 payload = 2 [(scalapb.field).no_box = true]; +} + +message PythonActorMessage { + common.ActorCommand payload = 1 [(scalapb.field).no_box = true]; +} \ No newline at end of file diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/control_commands.proto similarity index 60% rename from core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto rename to core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/control_commands.proto index d6950465e1d..83797195162 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/control_commands.proto @@ -1,7 +1,7 @@ syntax = "proto3"; package edu.uci.ics.amber.engine.architecture.rpc; -import "edu/uci/ics/amber/engine/common/virtualidentity.proto"; +import "edu/uci/ics/amber/engine/common/virtual_identity.proto"; import "edu/uci/ics/amber/engine/common/workflow.proto"; import "edu/uci/ics/amber/engine/architecture/worker/statistics.proto"; import "edu/uci/ics/amber/engine/architecture/sendsemantics/partitionings.proto"; @@ -13,47 +13,49 @@ option (scalapb.options) = { scope: FILE, preserve_unknown_fields: false no_default_values_in_constructor: true + flat_package: true }; message ControlRequest { oneof sealed_value { // request for controller - PropagateChannelMarkerRequest propagateChannelMarkerRequest = 1; - TakeGlobalCheckpointRequest takeGlobalCheckpointRequest = 2; - DebugCommandRequest debugCommandRequest = 3; - EvaluatePythonExpressionRequest evaluatePythonExpressionRequest = 4; - ModifyLogicRequest modifyLogicRequest = 5; - RetryWorkflowRequest retryWorkflowRequest = 6; - ConsoleMessageTriggeredRequest consoleMessageTriggeredRequest = 8; - PortCompletedRequest portCompletedRequest = 9; - WorkerStateUpdatedRequest workerStateUpdatedRequest = 10; - LinkWorkersRequest linkWorkersRequest = 11; + PropagateChannelMarkerRequest propagate_channel_marker_request = 1; + TakeGlobalCheckpointRequest take_global_checkpoint_request = 2; + DebugCommandRequest debug_command_request = 3; + EvaluatePythonExpressionRequest evaluate_python_expression_request = 4; + ModifyLogicRequest modify_logic_request = 5; + RetryWorkflowRequest retry_workflow_request = 6; + ConsoleMessageTriggeredRequest console_message_triggered_request = 8; + PortCompletedRequest port_completed_request = 9; + WorkerStateUpdatedRequest worker_state_updated_request = 10; + LinkWorkersRequest link_workers_request = 11; // request for worker - AddInputChannelRequest addInputChannelRequest = 50; - AddPartitioningRequest addPartitioningRequest = 51; - AssignPortRequest assignPortRequest = 52; - FinalizeCheckpointRequest finalizeCheckpointRequest = 53; - InitializeExecutorRequest initializeExecutorRequest = 54; - UpdateExecutorRequest updateExecutorRequest = 55; - EmptyRequest emptyRequest = 56; - PrepareCheckpointRequest prepareCheckpointRequest = 57; - QueryStatisticsRequest queryStatisticsRequest = 58; + AddInputChannelRequest add_input_channel_request = 50; + AddPartitioningRequest add_partitioning_request = 51; + AssignPortRequest assign_port_request = 52; + FinalizeCheckpointRequest finalize_checkpoint_request = 53; + InitializeExecutorRequest initialize_executor_request = 54; + UpdateExecutorRequest update_executor_request = 55; + EmptyRequest empty_request = 56; + PrepareCheckpointRequest prepare_checkpoint_request = 57; + QueryStatisticsRequest query_statistics_request = 58; // request for testing Ping ping = 100; Pong pong = 101; Nested nested = 102; Pass pass = 103; - ErrorCommand errorCommand = 104; + ErrorCommand error_command = 104; Recursion recursion = 105; Collect collect = 106; - GenerateNumber generateNumber = 107; - MultiCall multiCall = 108; + GenerateNumber generate_number = 107; + MultiCall multi_call = 108; Chain chain = 109; } } + message EmptyRequest{} message AsyncRPCContext { @@ -64,10 +66,10 @@ message AsyncRPCContext { message ControlInvocation { option (scalapb.message).extends = "edu.uci.ics.amber.engine.common.ambermessage.ControlPayload"; - string methodName = 1; + string method_name = 1; ControlRequest command = 2 [(scalapb.field).no_box = true]; AsyncRPCContext context = 3; - int64 commandId = 4; + int64 command_id = 4; } // Enum for ChannelMarkerType @@ -80,45 +82,45 @@ enum ChannelMarkerType { message ChannelMarkerPayload { option (scalapb.message).extends = "edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessagePayload"; common.ChannelMarkerIdentity id = 1 [(scalapb.field).no_box = true]; - ChannelMarkerType markerType = 2; + ChannelMarkerType marker_type = 2; repeated common.ChannelIdentity scope = 3; - map commandMapping = 4; + map command_mapping = 4; } message PropagateChannelMarkerRequest { - repeated common.PhysicalOpIdentity sourceOpToStartProp = 1; + repeated common.PhysicalOpIdentity source_op_to_start_prop = 1; common.ChannelMarkerIdentity id = 2 [(scalapb.field).no_box = true]; - ChannelMarkerType markerType = 3; + ChannelMarkerType marker_type = 3; repeated common.PhysicalOpIdentity scope = 4; - repeated common.PhysicalOpIdentity targetOps = 5; - ControlRequest markerCommand = 6; - string markerMethodName = 7; + repeated common.PhysicalOpIdentity target_ops = 5; + ControlRequest marker_command = 6; + string marker_method_name = 7; } message TakeGlobalCheckpointRequest { - bool estimationOnly = 1; - common.ChannelMarkerIdentity checkpointId = 2 [(scalapb.field).no_box = true]; + bool estimation_only = 1; + common.ChannelMarkerIdentity checkpoint_id = 2 [(scalapb.field).no_box = true]; string destination = 3; } message WorkflowReconfigureRequest{ ModifyLogicRequest reconfiguration = 1 [(scalapb.field).no_box = true]; - string reconfigurationId = 2; + string reconfiguration_id = 2; } message DebugCommandRequest { - string workerId = 1; + string worker_id = 1; string cmd = 2; } message EvaluatePythonExpressionRequest { string expression = 1; - string operatorId = 2; + string operator_id = 2; } message ModifyLogicRequest { - repeated UpdateExecutorRequest updateRequest = 1; + repeated UpdateExecutorRequest update_request = 1; } message RetryWorkflowRequest { @@ -143,11 +145,11 @@ message ConsoleMessage { } message ConsoleMessageTriggeredRequest { - ConsoleMessage consoleMessage = 1 [(scalapb.field).no_box = true]; + ConsoleMessage console_message = 1 [(scalapb.field).no_box = true]; } message PortCompletedRequest { - common.PortIdentity portId = 1 [(scalapb.field).no_box = true]; + common.PortIdentity port_id = 1 [(scalapb.field).no_box = true]; bool input = 2; } @@ -213,8 +215,8 @@ message Recursion { // Messages for the commands message AddInputChannelRequest { - common.ChannelIdentity channelId = 1 [(scalapb.field).no_box = true]; - common.PortIdentity portId = 2 [(scalapb.field).no_box = true]; + common.ChannelIdentity channel_id = 1 [(scalapb.field).no_box = true]; + common.PortIdentity port_id = 2 [(scalapb.field).no_box = true]; } message AddPartitioningRequest { @@ -223,34 +225,34 @@ message AddPartitioningRequest { } message AssignPortRequest { - common.PortIdentity portId = 1 [(scalapb.field).no_box = true]; + common.PortIdentity port_id = 1 [(scalapb.field).no_box = true]; bool input = 2; map schema = 3; } message FinalizeCheckpointRequest { - common.ChannelMarkerIdentity checkpointId = 1 [(scalapb.field).no_box = true]; - string writeTo = 2; + common.ChannelMarkerIdentity checkpoint_id = 1 [(scalapb.field).no_box = true]; + string write_to = 2; } message InitializeExecutorRequest { - int32 totalWorkerCount = 1; - google.protobuf.Any opExecInitInfo = 2 [(scalapb.field).no_box = true]; - bool isSource = 3; + int32 total_worker_count = 1; + google.protobuf.Any op_exec_init_info = 2 [(scalapb.field).no_box = true]; + bool is_source = 3; string language = 4; } message UpdateExecutorRequest { - common.PhysicalOpIdentity targetOpId = 1 [(scalapb.field).no_box = true]; - google.protobuf.Any newExecutor = 2 [(scalapb.field).no_box = true]; - google.protobuf.Any stateTransferFunc = 3; + common.PhysicalOpIdentity target_op_id = 1 [(scalapb.field).no_box = true]; + google.protobuf.Any new_executor = 2 [(scalapb.field).no_box = true]; + google.protobuf.Any state_transfer_func = 3; } message PrepareCheckpointRequest{ - common.ChannelMarkerIdentity checkpointId = 1 [(scalapb.field).no_box = true]; - bool estimationOnly = 2; + common.ChannelMarkerIdentity checkpoint_id = 1 [(scalapb.field).no_box = true]; + bool estimation_only = 2; } message QueryStatisticsRequest{ - repeated common.ActorVirtualIdentity filterByWorkers = 1; + repeated common.ActorVirtualIdentity filter_by_workers = 1; } \ No newline at end of file diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/control_returns.proto similarity index 66% rename from core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto rename to core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/control_returns.proto index 85cdb137237..a77f16f8188 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/control_returns.proto @@ -2,12 +2,14 @@ syntax = "proto3"; package edu.uci.ics.amber.engine.architecture.rpc; import "edu/uci/ics/amber/engine/architecture/worker/statistics.proto"; +import "edu/uci/ics/amber/engine/common/workflow_metrics.proto"; import "scalapb/scalapb.proto"; option (scalapb.options) = { scope: FILE, preserve_unknown_fields: false no_default_values_in_constructor: true + flat_package: true }; @@ -16,22 +18,22 @@ message ControlReturn { // Oneof block for various return types oneof sealed_value { // controller responses - RetrieveWorkflowStateResponse retrieveWorkflowStateResponse = 1; - PropagateChannelMarkerResponse propagateChannelMarkerResponse = 2; - TakeGlobalCheckpointResponse takeGlobalCheckpointResponse = 3; - EvaluatePythonExpressionResponse evaluatePythonExpressionResponse = 4; - StartWorkflowResponse startWorkflowResponse = 5; + RetrieveWorkflowStateResponse retrieve_workflow_state_response = 1; + PropagateChannelMarkerResponse propagate_channel_marker_response = 2; + TakeGlobalCheckpointResponse take_global_checkpoint_response = 3; + EvaluatePythonExpressionResponse evaluate_python_expression_response = 4; + StartWorkflowResponse start_workflow_response = 5; // worker responses - WorkerStateResponse workerStateResponse = 50; - WorkerMetricsResponse workerMetricsResponse = 51; - FinalizeCheckpointResponse finalizeCheckpointResponse = 52; + WorkerStateResponse worker_state_response = 50; + WorkerMetricsResponse worker_metrics_response = 51; + FinalizeCheckpointResponse finalize_checkpoint_response = 52; // common responses - ControlError error = 101; - EmptyReturn emptyReturn = 102; - StringResponse stringResponse = 103; - IntResponse intResponse = 104; + ControlException error = 101; + EmptyReturn empty_return = 102; + StringResponse string_response = 103; + IntResponse int_response = 104; } } @@ -42,7 +44,7 @@ enum ErrorLanguage { SCALA = 1; } -message ControlError { +message ControlException { string errorMessage = 1; string errorDetails = 2; string stackTrace = 3; @@ -97,21 +99,8 @@ message EvaluatePythonExpressionResponse { repeated EvaluatedValue values = 1; } -enum WorkflowAggregatedState { - UNINITIALIZED = 0; - READY = 1; - RUNNING = 2; - PAUSING = 3; - PAUSED = 4; - RESUMING = 5; - COMPLETED = 6; - FAILED = 7; - UNKNOWN = 8; - KILLED = 9; -} - message StartWorkflowResponse { - WorkflowAggregatedState workflowState = 1 [(scalapb.field).no_box = true]; + common.WorkflowAggregatedState workflow_state = 1 [(scalapb.field).no_box = true]; } message WorkerStateResponse { diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controllerservice.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controller_service.proto similarity index 89% rename from core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controllerservice.proto rename to core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controller_service.proto index de4a1e6d38d..89847ed6476 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controllerservice.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controller_service.proto @@ -1,14 +1,15 @@ syntax = "proto3"; package edu.uci.ics.amber.engine.architecture.rpc; -import "edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto"; -import "edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto"; +import "edu/uci/ics/amber/engine/architecture/rpc/control_commands.proto"; +import "edu/uci/ics/amber/engine/architecture/rpc/control_returns.proto"; import "scalapb/scalapb.proto"; option (scalapb.options) = { scope: FILE, preserve_unknown_fields: false no_default_values_in_constructor: true + flat_package: true }; diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/testerservice.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/tester_service.proto similarity index 83% rename from core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/testerservice.proto rename to core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/tester_service.proto index 2e131090180..79f26b34bad 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/testerservice.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/tester_service.proto @@ -1,14 +1,15 @@ syntax = "proto3"; package edu.uci.ics.amber.engine.architecture.rpc; -import "edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto"; -import "edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto"; +import "edu/uci/ics/amber/engine/architecture/rpc/control_commands.proto"; +import "edu/uci/ics/amber/engine/architecture/rpc/control_returns.proto"; import "scalapb/scalapb.proto"; option (scalapb.options) = { scope: FILE, preserve_unknown_fields: false no_default_values_in_constructor: true + flat_package: true }; diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/workerservice.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/worker_service.proto similarity index 89% rename from core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/workerservice.proto rename to core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/worker_service.proto index d24a855c476..3a4a148d79a 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/workerservice.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/worker_service.proto @@ -1,14 +1,15 @@ syntax = "proto3"; package edu.uci.ics.amber.engine.architecture.rpc; -import "edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto"; -import "edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto"; +import "edu/uci/ics/amber/engine/architecture/rpc/control_commands.proto"; +import "edu/uci/ics/amber/engine/architecture/rpc/control_returns.proto"; import "scalapb/scalapb.proto"; option (scalapb.options) = { scope: FILE, preserve_unknown_fields: false no_default_values_in_constructor: true + flat_package: true }; // RPC Service diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/sendsemantics/partitionings.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/sendsemantics/partitionings.proto index 5cfefb23905..3daba31ec04 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/sendsemantics/partitionings.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/sendsemantics/partitionings.proto @@ -8,45 +8,47 @@ option (scalapb.options) = { scope: FILE, preserve_unknown_fields: false no_default_values_in_constructor: true + flat_package: true }; -import "edu/uci/ics/amber/engine/common/virtualidentity.proto"; +import "edu/uci/ics/amber/engine/common/virtual_identity.proto"; -message Partitioning{ - oneof sealed_value{ - OneToOnePartitioning oneToOnePartitioning = 1; - RoundRobinPartitioning roundRobinPartitioning = 2; - HashBasedShufflePartitioning hashBasedShufflePartitioning = 3; - RangeBasedShufflePartitioning rangeBasedShufflePartitioning = 4; - BroadcastPartitioning broadcastPartitioning = 5; +message Partitioning { + oneof sealed_value { + OneToOnePartitioning one_to_one_partitioning = 1; + RoundRobinPartitioning round_robin_partitioning = 2; + HashBasedShufflePartitioning hash_based_shuffle_partitioning = 3; + RangeBasedShufflePartitioning range_based_shuffle_partitioning = 4; + BroadcastPartitioning broadcast_partitioning = 5; } } + message OneToOnePartitioning{ - int32 batchSize = 1; + int32 batch_size = 1; repeated edu.uci.ics.amber.engine.common.ChannelIdentity channels = 2; } message RoundRobinPartitioning{ - int32 batchSize = 1; + int32 batch_size = 1; repeated edu.uci.ics.amber.engine.common.ChannelIdentity channels = 2; } message HashBasedShufflePartitioning{ - int32 batchSize = 1; + int32 batch_size = 1; repeated edu.uci.ics.amber.engine.common.ChannelIdentity channels = 2; - repeated string hashAttributeNames = 3; + repeated string hash_attribute_names = 3; } message RangeBasedShufflePartitioning { - int32 batchSize = 1; + int32 batch_size = 1; repeated edu.uci.ics.amber.engine.common.ChannelIdentity channels = 2; - repeated string rangeAttributeNames = 3; - int64 rangeMin = 4; - int64 rangeMax = 5; + repeated string range_attribute_names = 3; + int64 range_min = 4; + int64 range_max = 5; } message BroadcastPartitioning{ - int32 batchSize = 1; + int32 batch_size = 1; repeated edu.uci.ics.amber.engine.common.ChannelIdentity channels = 2; } diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/worker/statistics.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/worker/statistics.proto index bc6e4e25fa3..519a47d5d0c 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/worker/statistics.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/worker/statistics.proto @@ -9,7 +9,7 @@ option (scalapb.options) = { scope: FILE, preserve_unknown_fields: false no_default_values_in_constructor: true - + flat_package: true }; enum WorkerState { diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/actormessage.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/actor_message.proto similarity index 69% rename from core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/actormessage.proto rename to core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/actor_message.proto index 1e9b9a56337..ebc8f778f72 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/actormessage.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/actor_message.proto @@ -9,10 +9,11 @@ option (scalapb.options) = { scope: FILE, preserve_unknown_fields: false no_default_values_in_constructor: true + flat_package: true }; message Backpressure { - bool enableBackpressure = 1; + bool enable_backpressure = 1; } message CreditUpdate { @@ -21,10 +22,6 @@ message CreditUpdate { message ActorCommand { oneof sealed_value { Backpressure backpressure = 1; - CreditUpdate creditUpdate = 2; + CreditUpdate credit_update = 2; } -} - -message PythonActorMessage { - ActorCommand payload = 1 [(scalapb.field).no_box = true]; } \ No newline at end of file diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/ambermessage.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/ambermessage.proto deleted file mode 100644 index 5d18f2b38b7..00000000000 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/ambermessage.proto +++ /dev/null @@ -1,31 +0,0 @@ -syntax = "proto3"; - -package edu.uci.ics.amber.engine.common; - -import "edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto"; -import "edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto"; -import "edu/uci/ics/amber/engine/common/virtualidentity.proto"; -import "scalapb/scalapb.proto"; - -option (scalapb.options) = { - scope: FILE, - preserve_unknown_fields: false - no_default_values_in_constructor: true -}; - -message ControlPayloadV2 { - oneof value { - architecture.rpc.ControlInvocation control_invocation = 1; - architecture.rpc.ReturnInvocation return_invocation = 2; - } -} - -message PythonDataHeader { - common.ActorVirtualIdentity tag = 1 [(scalapb.field).no_box = true]; - string payload_type = 2; -} - -message PythonControlMessage { - common.ActorVirtualIdentity tag = 1 [(scalapb.field).no_box = true]; - ControlPayloadV2 payload = 2 [(scalapb.field).no_box = true]; -} diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/virtualidentity.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/virtual_identity.proto similarity index 65% rename from core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/virtualidentity.proto rename to core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/virtual_identity.proto index df26635443b..b8d00b7365d 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/virtualidentity.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/virtual_identity.proto @@ -8,6 +8,7 @@ option (scalapb.options) = { scope: FILE, preserve_unknown_fields: false no_default_values_in_constructor: true + flat_package: true }; message WorkflowIdentity { @@ -23,9 +24,9 @@ message ActorVirtualIdentity { } message ChannelIdentity { - ActorVirtualIdentity fromWorkerId = 1 [(scalapb.field).no_box = true]; - ActorVirtualIdentity toWorkerId = 2 [(scalapb.field).no_box = true]; - bool isControl = 3; + ActorVirtualIdentity from_worker_id = 1 [(scalapb.field).no_box = true]; + ActorVirtualIdentity to_worker_id = 2 [(scalapb.field).no_box = true]; + bool is_control = 3; } message OperatorIdentity { @@ -33,8 +34,8 @@ message OperatorIdentity { } message PhysicalOpIdentity{ - OperatorIdentity logicalOpId = 1 [(scalapb.field).no_box = true]; - string layerName = 2; + OperatorIdentity logical_op_id = 1 [(scalapb.field).no_box = true]; + string layer_name = 2; } message ChannelMarkerIdentity{ diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflow.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflow.proto index 5d86ee18728..3a2ab3548dc 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflow.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflow.proto @@ -2,13 +2,14 @@ syntax = "proto3"; package edu.uci.ics.amber.engine.common; -import "edu/uci/ics/amber/engine/common/virtualidentity.proto"; +import "edu/uci/ics/amber/engine/common/virtual_identity.proto"; import "scalapb/scalapb.proto"; option (scalapb.options) = { scope: FILE, preserve_unknown_fields: false, no_default_values_in_constructor: false + flat_package: true }; message PortIdentity { @@ -18,21 +19,21 @@ message PortIdentity { message InputPort { PortIdentity id = 1 [(scalapb.field).no_box = true]; - string displayName = 2; - bool allowMultiLinks = 3; + string display_name = 2; + bool allow_multi_links = 3; repeated PortIdentity dependencies = 4; } message OutputPort { PortIdentity id = 1 [(scalapb.field).no_box = true]; - string displayName = 2; + string display_name = 2; bool blocking = 3; } message PhysicalLink { - PhysicalOpIdentity fromOpId = 1 [(scalapb.field).no_box = true]; - PortIdentity fromPortId = 2 [(scalapb.field).no_box = true]; - PhysicalOpIdentity toOpId = 3 [(scalapb.field).no_box = true]; - PortIdentity toPortId = 4 [(scalapb.field).no_box = true]; + PhysicalOpIdentity from_op_id = 1 [(scalapb.field).no_box = true]; + PortIdentity from_port_id = 2 [(scalapb.field).no_box = true]; + PhysicalOpIdentity to_op_id = 3 [(scalapb.field).no_box = true]; + PortIdentity to_port_id = 4 [(scalapb.field).no_box = true]; } diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflow_metrics.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflow_metrics.proto new file mode 100644 index 00000000000..b99ed47baab --- /dev/null +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflow_metrics.proto @@ -0,0 +1,41 @@ +syntax = "proto3"; + +package edu.uci.ics.amber.engine.common; + +import "edu/uci/ics/amber/engine/architecture/worker/statistics.proto"; +import "edu/uci/ics/amber/engine/common/virtual_identity.proto"; +import "scalapb/scalapb.proto"; + +option (scalapb.options) = { + scope: FILE, + preserve_unknown_fields: false + no_default_values_in_constructor: false + flat_package: true +}; + +enum WorkflowAggregatedState { + UNINITIALIZED = 0; + READY = 1; + RUNNING = 2; + PAUSING = 3; + PAUSED = 4; + RESUMING = 5; + COMPLETED = 6; + FAILED = 7; + UNKNOWN = 8; + KILLED = 9; +} + +message OperatorStatistics{ + repeated amber.engine.architecture.worker.PortTupleCountMapping input_count = 1; + repeated amber.engine.architecture.worker.PortTupleCountMapping output_count = 2; + int32 num_workers = 3; + int64 data_processing_time = 4; + int64 control_processing_time = 5; + int64 idle_time = 6; +} + +message OperatorMetrics{ + WorkflowAggregatedState operator_state = 1 [(scalapb.field).no_box = true]; + OperatorStatistics operator_statistics = 2 [(scalapb.field).no_box = true]; +} diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflowruntimestate.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflow_runtime_state.proto similarity index 57% rename from core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflowruntimestate.proto rename to core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflow_runtime_state.proto index b0c9edda839..08b3d8481c0 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflowruntimestate.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/workflow_runtime_state.proto @@ -2,10 +2,10 @@ syntax = "proto3"; package edu.uci.ics.amber.engine.common; -import "edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto"; -import "edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto"; -import "edu/uci/ics/amber/engine/architecture/worker/statistics.proto"; -import "edu/uci/ics/amber/engine/common/virtualidentity.proto"; +import "edu/uci/ics/amber/engine/architecture/rpc/control_commands.proto"; +import "edu/uci/ics/amber/engine/architecture/rpc/control_returns.proto"; +import "edu/uci/ics/amber/engine/common/workflow_metrics.proto"; +import "edu/uci/ics/amber/engine/common/virtual_identity.proto"; import "google/protobuf/timestamp.proto"; import "scalapb/scalapb.proto"; @@ -13,6 +13,7 @@ option (scalapb.options) = { scope: FILE, preserve_unknown_fields: false no_default_values_in_constructor: false + flat_package: true }; enum FatalErrorType{ @@ -20,7 +21,6 @@ enum FatalErrorType{ EXECUTION_FAILURE = 1; } - message BreakpointFault{ message BreakpointTuple{ int64 id = 1; @@ -54,44 +54,29 @@ message ExecutionConsoleStore{ } message OperatorWorkerMapping{ - string operatorId = 1; - repeated string workerIds = 2; -} - -message OperatorStatistics{ - repeated amber.engine.architecture.worker.PortTupleCountMapping input_count = 1; - repeated amber.engine.architecture.worker.PortTupleCountMapping output_count = 2; - int32 num_workers = 3; - int64 data_processing_time = 4; - int64 control_processing_time = 5; - int64 idle_time = 6; -} - -message OperatorMetrics{ - edu.uci.ics.amber.engine.architecture.rpc.WorkflowAggregatedState operator_state = 1 [(scalapb.field).no_box = true]; - OperatorStatistics operator_statistics = 2 [(scalapb.field).no_box = true]; + string operator_id = 1; + repeated string worker_ids = 2; } message ExecutionStatsStore { - int64 startTimeStamp = 1; - int64 endTimeStamp = 2; + int64 start_timestamp = 1; + int64 end_timestamp = 2; map operator_info = 3; repeated OperatorWorkerMapping operator_worker_mapping = 4; } - message WorkflowFatalError { FatalErrorType type = 1; google.protobuf.Timestamp timestamp = 2 [(scalapb.field).no_box = true]; string message = 3; string details = 4; - string operatorId = 5; - string workerId = 6; + string operator_id = 5; + string worker_id = 6; } message ExecutionMetadataStore{ - edu.uci.ics.amber.engine.architecture.rpc.WorkflowAggregatedState state = 1; + WorkflowAggregatedState state = 1; repeated WorkflowFatalError fatal_errors = 2; - edu.uci.ics.amber.engine.common.ExecutionIdentity executionId = 3 [(scalapb.field).no_box = true]; + edu.uci.ics.amber.engine.common.ExecutionIdentity execution_id = 3 [(scalapb.field).no_box = true]; bool is_recovering = 4; } diff --git a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/rpc/__init__.py b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/rpc/__init__.py index e5e19d7e087..12f9c9b30d4 100644 --- a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/rpc/__init__.py +++ b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/rpc/__init__.py @@ -1,5 +1,5 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! -# sources: edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto, edu/uci/ics/amber/engine/architecture/rpc/controllerservice.proto, edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto, edu/uci/ics/amber/engine/architecture/rpc/testerservice.proto, edu/uci/ics/amber/engine/architecture/rpc/workerservice.proto +# sources: edu/uci/ics/amber/engine/architecture/rpc/control_commands.proto, edu/uci/ics/amber/engine/architecture/rpc/control_returns.proto, edu/uci/ics/amber/engine/architecture/rpc/controller_service.proto, edu/uci/ics/amber/engine/architecture/rpc/tester_service.proto, edu/uci/ics/amber/engine/architecture/rpc/worker_service.proto # plugin: python-betterproto # This file has been @generated @@ -49,19 +49,6 @@ class ErrorLanguage(betterproto.Enum): SCALA = 1 -class WorkflowAggregatedState(betterproto.Enum): - UNINITIALIZED = 0 - READY = 1 - RUNNING = 2 - PAUSING = 3 - PAUSED = 4 - RESUMING = 5 - COMPLETED = 6 - FAILED = 7 - UNKNOWN = 8 - KILLED = 9 - - @dataclass(eq=False, repr=False) class ControlRequest(betterproto.Message): propagate_channel_marker_request: "PropagateChannelMarkerRequest" = ( @@ -416,7 +403,7 @@ class ControlReturn(betterproto.Message): finalize_checkpoint_response: "FinalizeCheckpointResponse" = ( betterproto.message_field(52, group="sealed_value") ) - error: "ControlError" = betterproto.message_field(101, group="sealed_value") + error: "ControlException" = betterproto.message_field(101, group="sealed_value") """common responses""" empty_return: "EmptyReturn" = betterproto.message_field(102, group="sealed_value") @@ -432,7 +419,7 @@ class EmptyReturn(betterproto.Message): @dataclass(eq=False, repr=False) -class ControlError(betterproto.Message): +class ControlException(betterproto.Message): error_message: str = betterproto.string_field(1) error_details: str = betterproto.string_field(2) stack_trace: str = betterproto.string_field(3) @@ -501,7 +488,7 @@ class EvaluatePythonExpressionResponse(betterproto.Message): @dataclass(eq=False, repr=False) class StartWorkflowResponse(betterproto.Message): - workflow_state: "WorkflowAggregatedState" = betterproto.enum_field(1) + workflow_state: "__common__.WorkflowAggregatedState" = betterproto.enum_field(1) @dataclass(eq=False, repr=False) @@ -514,178 +501,6 @@ class WorkerMetricsResponse(betterproto.Message): metrics: "_worker__.WorkerMetrics" = betterproto.message_field(1) -class RpcTesterStub(betterproto.ServiceStub): - async def send_ping( - self, - ping: "Ping", - *, - timeout: Optional[float] = None, - deadline: Optional["Deadline"] = None, - metadata: Optional["MetadataLike"] = None - ) -> "IntResponse": - return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendPing", - ping, - IntResponse, - timeout=timeout, - deadline=deadline, - metadata=metadata, - ) - - async def send_pong( - self, - pong: "Pong", - *, - timeout: Optional[float] = None, - deadline: Optional["Deadline"] = None, - metadata: Optional["MetadataLike"] = None - ) -> "IntResponse": - return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendPong", - pong, - IntResponse, - timeout=timeout, - deadline=deadline, - metadata=metadata, - ) - - async def send_nested( - self, - nested: "Nested", - *, - timeout: Optional[float] = None, - deadline: Optional["Deadline"] = None, - metadata: Optional["MetadataLike"] = None - ) -> "StringResponse": - return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendNested", - nested, - StringResponse, - timeout=timeout, - deadline=deadline, - metadata=metadata, - ) - - async def send_pass( - self, - pass_: "Pass", - *, - timeout: Optional[float] = None, - deadline: Optional["Deadline"] = None, - metadata: Optional["MetadataLike"] = None - ) -> "StringResponse": - return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendPass", - pass_, - StringResponse, - timeout=timeout, - deadline=deadline, - metadata=metadata, - ) - - async def send_error_command( - self, - error_command: "ErrorCommand", - *, - timeout: Optional[float] = None, - deadline: Optional["Deadline"] = None, - metadata: Optional["MetadataLike"] = None - ) -> "StringResponse": - return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendErrorCommand", - error_command, - StringResponse, - timeout=timeout, - deadline=deadline, - metadata=metadata, - ) - - async def send_recursion( - self, - recursion: "Recursion", - *, - timeout: Optional[float] = None, - deadline: Optional["Deadline"] = None, - metadata: Optional["MetadataLike"] = None - ) -> "StringResponse": - return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendRecursion", - recursion, - StringResponse, - timeout=timeout, - deadline=deadline, - metadata=metadata, - ) - - async def send_collect( - self, - collect: "Collect", - *, - timeout: Optional[float] = None, - deadline: Optional["Deadline"] = None, - metadata: Optional["MetadataLike"] = None - ) -> "StringResponse": - return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendCollect", - collect, - StringResponse, - timeout=timeout, - deadline=deadline, - metadata=metadata, - ) - - async def send_generate_number( - self, - generate_number: "GenerateNumber", - *, - timeout: Optional[float] = None, - deadline: Optional["Deadline"] = None, - metadata: Optional["MetadataLike"] = None - ) -> "IntResponse": - return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendGenerateNumber", - generate_number, - IntResponse, - timeout=timeout, - deadline=deadline, - metadata=metadata, - ) - - async def send_multi_call( - self, - multi_call: "MultiCall", - *, - timeout: Optional[float] = None, - deadline: Optional["Deadline"] = None, - metadata: Optional["MetadataLike"] = None - ) -> "StringResponse": - return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendMultiCall", - multi_call, - StringResponse, - timeout=timeout, - deadline=deadline, - metadata=metadata, - ) - - async def send_chain( - self, - chain: "Chain", - *, - timeout: Optional[float] = None, - deadline: Optional["Deadline"] = None, - metadata: Optional["MetadataLike"] = None - ) -> "StringResponse": - return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendChain", - chain, - StringResponse, - timeout=timeout, - deadline=deadline, - metadata=metadata, - ) - - class WorkerServiceStub(betterproto.ServiceStub): async def add_input_channel( self, @@ -977,120 +792,292 @@ async def no_operation( ) -class ControllerServiceStub(betterproto.ServiceStub): - async def retrieve_workflow_state( +class RpcTesterStub(betterproto.ServiceStub): + async def send_ping( self, - empty_request: "EmptyRequest", + ping: "Ping", *, timeout: Optional[float] = None, deadline: Optional["Deadline"] = None, metadata: Optional["MetadataLike"] = None - ) -> "RetrieveWorkflowStateResponse": + ) -> "IntResponse": return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/RetrieveWorkflowState", - empty_request, - RetrieveWorkflowStateResponse, + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendPing", + ping, + IntResponse, timeout=timeout, deadline=deadline, metadata=metadata, ) - async def propagate_channel_marker( + async def send_pong( self, - propagate_channel_marker_request: "PropagateChannelMarkerRequest", + pong: "Pong", *, timeout: Optional[float] = None, deadline: Optional["Deadline"] = None, metadata: Optional["MetadataLike"] = None - ) -> "PropagateChannelMarkerResponse": + ) -> "IntResponse": return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/PropagateChannelMarker", - propagate_channel_marker_request, - PropagateChannelMarkerResponse, + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendPong", + pong, + IntResponse, timeout=timeout, deadline=deadline, metadata=metadata, ) - async def take_global_checkpoint( + async def send_nested( self, - take_global_checkpoint_request: "TakeGlobalCheckpointRequest", + nested: "Nested", *, timeout: Optional[float] = None, deadline: Optional["Deadline"] = None, metadata: Optional["MetadataLike"] = None - ) -> "TakeGlobalCheckpointResponse": + ) -> "StringResponse": return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/TakeGlobalCheckpoint", - take_global_checkpoint_request, - TakeGlobalCheckpointResponse, + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendNested", + nested, + StringResponse, timeout=timeout, deadline=deadline, metadata=metadata, ) - async def debug_command( + async def send_pass( self, - debug_command_request: "DebugCommandRequest", + pass_: "Pass", *, timeout: Optional[float] = None, deadline: Optional["Deadline"] = None, metadata: Optional["MetadataLike"] = None - ) -> "EmptyReturn": + ) -> "StringResponse": return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/DebugCommand", - debug_command_request, - EmptyReturn, + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendPass", + pass_, + StringResponse, timeout=timeout, deadline=deadline, metadata=metadata, ) - async def evaluate_python_expression( + async def send_error_command( self, - evaluate_python_expression_request: "EvaluatePythonExpressionRequest", + error_command: "ErrorCommand", *, timeout: Optional[float] = None, deadline: Optional["Deadline"] = None, metadata: Optional["MetadataLike"] = None - ) -> "EvaluatePythonExpressionResponse": + ) -> "StringResponse": return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/EvaluatePythonExpression", - evaluate_python_expression_request, - EvaluatePythonExpressionResponse, + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendErrorCommand", + error_command, + StringResponse, timeout=timeout, deadline=deadline, metadata=metadata, ) - async def console_message_triggered( + async def send_recursion( self, - console_message_triggered_request: "ConsoleMessageTriggeredRequest", + recursion: "Recursion", *, timeout: Optional[float] = None, deadline: Optional["Deadline"] = None, metadata: Optional["MetadataLike"] = None - ) -> "EmptyReturn": + ) -> "StringResponse": return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/ConsoleMessageTriggered", - console_message_triggered_request, - EmptyReturn, + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendRecursion", + recursion, + StringResponse, timeout=timeout, deadline=deadline, metadata=metadata, ) - async def port_completed( + async def send_collect( self, - port_completed_request: "PortCompletedRequest", + collect: "Collect", *, timeout: Optional[float] = None, deadline: Optional["Deadline"] = None, metadata: Optional["MetadataLike"] = None - ) -> "EmptyReturn": + ) -> "StringResponse": return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/PortCompleted", - port_completed_request, + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendCollect", + collect, + StringResponse, + timeout=timeout, + deadline=deadline, + metadata=metadata, + ) + + async def send_generate_number( + self, + generate_number: "GenerateNumber", + *, + timeout: Optional[float] = None, + deadline: Optional["Deadline"] = None, + metadata: Optional["MetadataLike"] = None + ) -> "IntResponse": + return await self._unary_unary( + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendGenerateNumber", + generate_number, + IntResponse, + timeout=timeout, + deadline=deadline, + metadata=metadata, + ) + + async def send_multi_call( + self, + multi_call: "MultiCall", + *, + timeout: Optional[float] = None, + deadline: Optional["Deadline"] = None, + metadata: Optional["MetadataLike"] = None + ) -> "StringResponse": + return await self._unary_unary( + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendMultiCall", + multi_call, + StringResponse, + timeout=timeout, + deadline=deadline, + metadata=metadata, + ) + + async def send_chain( + self, + chain: "Chain", + *, + timeout: Optional[float] = None, + deadline: Optional["Deadline"] = None, + metadata: Optional["MetadataLike"] = None + ) -> "StringResponse": + return await self._unary_unary( + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendChain", + chain, + StringResponse, + timeout=timeout, + deadline=deadline, + metadata=metadata, + ) + + +class ControllerServiceStub(betterproto.ServiceStub): + async def retrieve_workflow_state( + self, + empty_request: "EmptyRequest", + *, + timeout: Optional[float] = None, + deadline: Optional["Deadline"] = None, + metadata: Optional["MetadataLike"] = None + ) -> "RetrieveWorkflowStateResponse": + return await self._unary_unary( + "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/RetrieveWorkflowState", + empty_request, + RetrieveWorkflowStateResponse, + timeout=timeout, + deadline=deadline, + metadata=metadata, + ) + + async def propagate_channel_marker( + self, + propagate_channel_marker_request: "PropagateChannelMarkerRequest", + *, + timeout: Optional[float] = None, + deadline: Optional["Deadline"] = None, + metadata: Optional["MetadataLike"] = None + ) -> "PropagateChannelMarkerResponse": + return await self._unary_unary( + "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/PropagateChannelMarker", + propagate_channel_marker_request, + PropagateChannelMarkerResponse, + timeout=timeout, + deadline=deadline, + metadata=metadata, + ) + + async def take_global_checkpoint( + self, + take_global_checkpoint_request: "TakeGlobalCheckpointRequest", + *, + timeout: Optional[float] = None, + deadline: Optional["Deadline"] = None, + metadata: Optional["MetadataLike"] = None + ) -> "TakeGlobalCheckpointResponse": + return await self._unary_unary( + "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/TakeGlobalCheckpoint", + take_global_checkpoint_request, + TakeGlobalCheckpointResponse, + timeout=timeout, + deadline=deadline, + metadata=metadata, + ) + + async def debug_command( + self, + debug_command_request: "DebugCommandRequest", + *, + timeout: Optional[float] = None, + deadline: Optional["Deadline"] = None, + metadata: Optional["MetadataLike"] = None + ) -> "EmptyReturn": + return await self._unary_unary( + "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/DebugCommand", + debug_command_request, + EmptyReturn, + timeout=timeout, + deadline=deadline, + metadata=metadata, + ) + + async def evaluate_python_expression( + self, + evaluate_python_expression_request: "EvaluatePythonExpressionRequest", + *, + timeout: Optional[float] = None, + deadline: Optional["Deadline"] = None, + metadata: Optional["MetadataLike"] = None + ) -> "EvaluatePythonExpressionResponse": + return await self._unary_unary( + "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/EvaluatePythonExpression", + evaluate_python_expression_request, + EvaluatePythonExpressionResponse, + timeout=timeout, + deadline=deadline, + metadata=metadata, + ) + + async def console_message_triggered( + self, + console_message_triggered_request: "ConsoleMessageTriggeredRequest", + *, + timeout: Optional[float] = None, + deadline: Optional["Deadline"] = None, + metadata: Optional["MetadataLike"] = None + ) -> "EmptyReturn": + return await self._unary_unary( + "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/ConsoleMessageTriggered", + console_message_triggered_request, + EmptyReturn, + timeout=timeout, + deadline=deadline, + metadata=metadata, + ) + + async def port_completed( + self, + port_completed_request: "PortCompletedRequest", + *, + timeout: Optional[float] = None, + deadline: Optional["Deadline"] = None, + metadata: Optional["MetadataLike"] = None + ) -> "EmptyReturn": + return await self._unary_unary( + "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/PortCompleted", + port_completed_request, EmptyReturn, timeout=timeout, deadline=deadline, @@ -1209,200 +1196,29 @@ async def controller_initiate_query_statistics( ) -> "EmptyReturn": return await self._unary_unary( "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/ControllerInitiateQueryStatistics", - query_statistics_request, - EmptyReturn, - timeout=timeout, - deadline=deadline, - metadata=metadata, - ) - - async def retry_workflow( - self, - retry_workflow_request: "RetryWorkflowRequest", - *, - timeout: Optional[float] = None, - deadline: Optional["Deadline"] = None, - metadata: Optional["MetadataLike"] = None - ) -> "EmptyReturn": - return await self._unary_unary( - "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/RetryWorkflow", - retry_workflow_request, - EmptyReturn, - timeout=timeout, - deadline=deadline, - metadata=metadata, - ) - - -class RpcTesterBase(ServiceBase): - - async def send_ping(self, ping: "Ping") -> "IntResponse": - raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) - - async def send_pong(self, pong: "Pong") -> "IntResponse": - raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) - - async def send_nested(self, nested: "Nested") -> "StringResponse": - raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) - - async def send_pass(self, pass_: "Pass") -> "StringResponse": - raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) - - async def send_error_command( - self, error_command: "ErrorCommand" - ) -> "StringResponse": - raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) - - async def send_recursion(self, recursion: "Recursion") -> "StringResponse": - raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) - - async def send_collect(self, collect: "Collect") -> "StringResponse": - raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) - - async def send_generate_number( - self, generate_number: "GenerateNumber" - ) -> "IntResponse": - raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) - - async def send_multi_call(self, multi_call: "MultiCall") -> "StringResponse": - raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) - - async def send_chain(self, chain: "Chain") -> "StringResponse": - raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) - - async def __rpc_send_ping( - self, stream: "grpclib.server.Stream[Ping, IntResponse]" - ) -> None: - request = await stream.recv_message() - response = await self.send_ping(request) - await stream.send_message(response) - - async def __rpc_send_pong( - self, stream: "grpclib.server.Stream[Pong, IntResponse]" - ) -> None: - request = await stream.recv_message() - response = await self.send_pong(request) - await stream.send_message(response) - - async def __rpc_send_nested( - self, stream: "grpclib.server.Stream[Nested, StringResponse]" - ) -> None: - request = await stream.recv_message() - response = await self.send_nested(request) - await stream.send_message(response) - - async def __rpc_send_pass( - self, stream: "grpclib.server.Stream[Pass, StringResponse]" - ) -> None: - request = await stream.recv_message() - response = await self.send_pass(request) - await stream.send_message(response) - - async def __rpc_send_error_command( - self, stream: "grpclib.server.Stream[ErrorCommand, StringResponse]" - ) -> None: - request = await stream.recv_message() - response = await self.send_error_command(request) - await stream.send_message(response) - - async def __rpc_send_recursion( - self, stream: "grpclib.server.Stream[Recursion, StringResponse]" - ) -> None: - request = await stream.recv_message() - response = await self.send_recursion(request) - await stream.send_message(response) - - async def __rpc_send_collect( - self, stream: "grpclib.server.Stream[Collect, StringResponse]" - ) -> None: - request = await stream.recv_message() - response = await self.send_collect(request) - await stream.send_message(response) - - async def __rpc_send_generate_number( - self, stream: "grpclib.server.Stream[GenerateNumber, IntResponse]" - ) -> None: - request = await stream.recv_message() - response = await self.send_generate_number(request) - await stream.send_message(response) - - async def __rpc_send_multi_call( - self, stream: "grpclib.server.Stream[MultiCall, StringResponse]" - ) -> None: - request = await stream.recv_message() - response = await self.send_multi_call(request) - await stream.send_message(response) - - async def __rpc_send_chain( - self, stream: "grpclib.server.Stream[Chain, StringResponse]" - ) -> None: - request = await stream.recv_message() - response = await self.send_chain(request) - await stream.send_message(response) + query_statistics_request, + EmptyReturn, + timeout=timeout, + deadline=deadline, + metadata=metadata, + ) - def __mapping__(self) -> Dict[str, grpclib.const.Handler]: - return { - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendPing": grpclib.const.Handler( - self.__rpc_send_ping, - grpclib.const.Cardinality.UNARY_UNARY, - Ping, - IntResponse, - ), - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendPong": grpclib.const.Handler( - self.__rpc_send_pong, - grpclib.const.Cardinality.UNARY_UNARY, - Pong, - IntResponse, - ), - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendNested": grpclib.const.Handler( - self.__rpc_send_nested, - grpclib.const.Cardinality.UNARY_UNARY, - Nested, - StringResponse, - ), - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendPass": grpclib.const.Handler( - self.__rpc_send_pass, - grpclib.const.Cardinality.UNARY_UNARY, - Pass, - StringResponse, - ), - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendErrorCommand": grpclib.const.Handler( - self.__rpc_send_error_command, - grpclib.const.Cardinality.UNARY_UNARY, - ErrorCommand, - StringResponse, - ), - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendRecursion": grpclib.const.Handler( - self.__rpc_send_recursion, - grpclib.const.Cardinality.UNARY_UNARY, - Recursion, - StringResponse, - ), - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendCollect": grpclib.const.Handler( - self.__rpc_send_collect, - grpclib.const.Cardinality.UNARY_UNARY, - Collect, - StringResponse, - ), - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendGenerateNumber": grpclib.const.Handler( - self.__rpc_send_generate_number, - grpclib.const.Cardinality.UNARY_UNARY, - GenerateNumber, - IntResponse, - ), - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendMultiCall": grpclib.const.Handler( - self.__rpc_send_multi_call, - grpclib.const.Cardinality.UNARY_UNARY, - MultiCall, - StringResponse, - ), - "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendChain": grpclib.const.Handler( - self.__rpc_send_chain, - grpclib.const.Cardinality.UNARY_UNARY, - Chain, - StringResponse, - ), - } + async def retry_workflow( + self, + retry_workflow_request: "RetryWorkflowRequest", + *, + timeout: Optional[float] = None, + deadline: Optional["Deadline"] = None, + metadata: Optional["MetadataLike"] = None + ) -> "EmptyReturn": + return await self._unary_unary( + "/edu.uci.ics.amber.engine.architecture.rpc.ControllerService/RetryWorkflow", + retry_workflow_request, + EmptyReturn, + timeout=timeout, + deadline=deadline, + metadata=metadata, + ) class WorkerServiceBase(ServiceBase): @@ -1712,6 +1528,177 @@ def __mapping__(self) -> Dict[str, grpclib.const.Handler]: } +class RpcTesterBase(ServiceBase): + + async def send_ping(self, ping: "Ping") -> "IntResponse": + raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) + + async def send_pong(self, pong: "Pong") -> "IntResponse": + raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) + + async def send_nested(self, nested: "Nested") -> "StringResponse": + raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) + + async def send_pass(self, pass_: "Pass") -> "StringResponse": + raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) + + async def send_error_command( + self, error_command: "ErrorCommand" + ) -> "StringResponse": + raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) + + async def send_recursion(self, recursion: "Recursion") -> "StringResponse": + raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) + + async def send_collect(self, collect: "Collect") -> "StringResponse": + raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) + + async def send_generate_number( + self, generate_number: "GenerateNumber" + ) -> "IntResponse": + raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) + + async def send_multi_call(self, multi_call: "MultiCall") -> "StringResponse": + raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) + + async def send_chain(self, chain: "Chain") -> "StringResponse": + raise grpclib.GRPCError(grpclib.const.Status.UNIMPLEMENTED) + + async def __rpc_send_ping( + self, stream: "grpclib.server.Stream[Ping, IntResponse]" + ) -> None: + request = await stream.recv_message() + response = await self.send_ping(request) + await stream.send_message(response) + + async def __rpc_send_pong( + self, stream: "grpclib.server.Stream[Pong, IntResponse]" + ) -> None: + request = await stream.recv_message() + response = await self.send_pong(request) + await stream.send_message(response) + + async def __rpc_send_nested( + self, stream: "grpclib.server.Stream[Nested, StringResponse]" + ) -> None: + request = await stream.recv_message() + response = await self.send_nested(request) + await stream.send_message(response) + + async def __rpc_send_pass( + self, stream: "grpclib.server.Stream[Pass, StringResponse]" + ) -> None: + request = await stream.recv_message() + response = await self.send_pass(request) + await stream.send_message(response) + + async def __rpc_send_error_command( + self, stream: "grpclib.server.Stream[ErrorCommand, StringResponse]" + ) -> None: + request = await stream.recv_message() + response = await self.send_error_command(request) + await stream.send_message(response) + + async def __rpc_send_recursion( + self, stream: "grpclib.server.Stream[Recursion, StringResponse]" + ) -> None: + request = await stream.recv_message() + response = await self.send_recursion(request) + await stream.send_message(response) + + async def __rpc_send_collect( + self, stream: "grpclib.server.Stream[Collect, StringResponse]" + ) -> None: + request = await stream.recv_message() + response = await self.send_collect(request) + await stream.send_message(response) + + async def __rpc_send_generate_number( + self, stream: "grpclib.server.Stream[GenerateNumber, IntResponse]" + ) -> None: + request = await stream.recv_message() + response = await self.send_generate_number(request) + await stream.send_message(response) + + async def __rpc_send_multi_call( + self, stream: "grpclib.server.Stream[MultiCall, StringResponse]" + ) -> None: + request = await stream.recv_message() + response = await self.send_multi_call(request) + await stream.send_message(response) + + async def __rpc_send_chain( + self, stream: "grpclib.server.Stream[Chain, StringResponse]" + ) -> None: + request = await stream.recv_message() + response = await self.send_chain(request) + await stream.send_message(response) + + def __mapping__(self) -> Dict[str, grpclib.const.Handler]: + return { + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendPing": grpclib.const.Handler( + self.__rpc_send_ping, + grpclib.const.Cardinality.UNARY_UNARY, + Ping, + IntResponse, + ), + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendPong": grpclib.const.Handler( + self.__rpc_send_pong, + grpclib.const.Cardinality.UNARY_UNARY, + Pong, + IntResponse, + ), + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendNested": grpclib.const.Handler( + self.__rpc_send_nested, + grpclib.const.Cardinality.UNARY_UNARY, + Nested, + StringResponse, + ), + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendPass": grpclib.const.Handler( + self.__rpc_send_pass, + grpclib.const.Cardinality.UNARY_UNARY, + Pass, + StringResponse, + ), + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendErrorCommand": grpclib.const.Handler( + self.__rpc_send_error_command, + grpclib.const.Cardinality.UNARY_UNARY, + ErrorCommand, + StringResponse, + ), + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendRecursion": grpclib.const.Handler( + self.__rpc_send_recursion, + grpclib.const.Cardinality.UNARY_UNARY, + Recursion, + StringResponse, + ), + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendCollect": grpclib.const.Handler( + self.__rpc_send_collect, + grpclib.const.Cardinality.UNARY_UNARY, + Collect, + StringResponse, + ), + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendGenerateNumber": grpclib.const.Handler( + self.__rpc_send_generate_number, + grpclib.const.Cardinality.UNARY_UNARY, + GenerateNumber, + IntResponse, + ), + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendMultiCall": grpclib.const.Handler( + self.__rpc_send_multi_call, + grpclib.const.Cardinality.UNARY_UNARY, + MultiCall, + StringResponse, + ), + "/edu.uci.ics.amber.engine.architecture.rpc.RPCTester/SendChain": grpclib.const.Handler( + self.__rpc_send_chain, + grpclib.const.Cardinality.UNARY_UNARY, + Chain, + StringResponse, + ), + } + + class ControllerServiceBase(ServiceBase): async def retrieve_workflow_state( diff --git a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/common/__init__.py b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/common/__init__.py index 1c38e3cc6cf..95781507601 100644 --- a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/common/__init__.py +++ b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/common/__init__.py @@ -1,5 +1,5 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! -# sources: edu/uci/ics/amber/engine/common/actormessage.proto, edu/uci/ics/amber/engine/common/ambermessage.proto, edu/uci/ics/amber/engine/common/virtualidentity.proto, edu/uci/ics/amber/engine/common/workflow.proto, edu/uci/ics/amber/engine/common/workflowruntimestate.proto +# sources: edu/uci/ics/amber/engine/common/actor_message.proto, edu/uci/ics/amber/engine/common/amber_message.proto, edu/uci/ics/amber/engine/common/virtual_identity.proto, edu/uci/ics/amber/engine/common/workflow.proto, edu/uci/ics/amber/engine/common/workflow_metrics.proto, edu/uci/ics/amber/engine/common/workflow_runtime_state.proto # plugin: python-betterproto # This file has been @generated @@ -18,6 +18,19 @@ ) +class WorkflowAggregatedState(betterproto.Enum): + UNINITIALIZED = 0 + READY = 1 + RUNNING = 2 + PAUSING = 3 + PAUSED = 4 + RESUMING = 5 + COMPLETED = 6 + FAILED = 7 + UNKNOWN = 8 + KILLED = 9 + + class FatalErrorType(betterproto.Enum): COMPILATION_ERROR = 0 EXECUTION_FAILURE = 1 @@ -90,6 +103,26 @@ class PhysicalLink(betterproto.Message): to_port_id: "PortIdentity" = betterproto.message_field(4) +@dataclass(eq=False, repr=False) +class OperatorStatistics(betterproto.Message): + input_count: List["_architecture_worker__.PortTupleCountMapping"] = ( + betterproto.message_field(1) + ) + output_count: List["_architecture_worker__.PortTupleCountMapping"] = ( + betterproto.message_field(2) + ) + num_workers: int = betterproto.int32_field(3) + data_processing_time: int = betterproto.int64_field(4) + control_processing_time: int = betterproto.int64_field(5) + idle_time: int = betterproto.int64_field(6) + + +@dataclass(eq=False, repr=False) +class OperatorMetrics(betterproto.Message): + operator_state: "WorkflowAggregatedState" = betterproto.enum_field(1) + operator_statistics: "OperatorStatistics" = betterproto.message_field(2) + + @dataclass(eq=False, repr=False) class ControlPayloadV2(betterproto.Message): control_invocation: "_architecture_rpc__.ControlInvocation" = ( @@ -112,6 +145,27 @@ class PythonControlMessage(betterproto.Message): payload: "ControlPayloadV2" = betterproto.message_field(2) +@dataclass(eq=False, repr=False) +class Backpressure(betterproto.Message): + enable_backpressure: bool = betterproto.bool_field(1) + + +@dataclass(eq=False, repr=False) +class CreditUpdate(betterproto.Message): + pass + + +@dataclass(eq=False, repr=False) +class ActorCommand(betterproto.Message): + backpressure: "Backpressure" = betterproto.message_field(1, group="sealed_value") + credit_update: "CreditUpdate" = betterproto.message_field(2, group="sealed_value") + + +@dataclass(eq=False, repr=False) +class PythonActorMessage(betterproto.Message): + payload: "ActorCommand" = betterproto.message_field(1) + + @dataclass(eq=False, repr=False) class BreakpointFault(betterproto.Message): worker_name: str = betterproto.string_field(1) @@ -165,40 +219,6 @@ class OperatorWorkerMapping(betterproto.Message): worker_ids: List[str] = betterproto.string_field(2) -@dataclass(eq=False, repr=False) -class OperatorStatistics(betterproto.Message): - input_count: List["_architecture_worker__.PortTupleCountMapping"] = ( - betterproto.message_field(1) - ) - output_count: List["_architecture_worker__.PortTupleCountMapping"] = ( - betterproto.message_field(2) - ) - num_workers: int = betterproto.int32_field(3) - data_processing_time: int = betterproto.int64_field(4) - control_processing_time: int = betterproto.int64_field(5) - idle_time: int = betterproto.int64_field(6) - - -@dataclass(eq=False, repr=False) -class OperatorMetrics(betterproto.Message): - operator_state: "_architecture_rpc__.WorkflowAggregatedState" = ( - betterproto.enum_field(1) - ) - operator_statistics: "OperatorStatistics" = betterproto.message_field(2) - - -@dataclass(eq=False, repr=False) -class ExecutionStatsStore(betterproto.Message): - start_time_stamp: int = betterproto.int64_field(1) - end_time_stamp: int = betterproto.int64_field(2) - operator_info: Dict[str, "OperatorMetrics"] = betterproto.map_field( - 3, betterproto.TYPE_STRING, betterproto.TYPE_MESSAGE - ) - operator_worker_mapping: List["OperatorWorkerMapping"] = betterproto.message_field( - 4 - ) - - @dataclass(eq=False, repr=False) class WorkflowFatalError(betterproto.Message): type: "FatalErrorType" = betterproto.enum_field(1) @@ -211,28 +231,7 @@ class WorkflowFatalError(betterproto.Message): @dataclass(eq=False, repr=False) class ExecutionMetadataStore(betterproto.Message): - state: "_architecture_rpc__.WorkflowAggregatedState" = betterproto.enum_field(1) + state: "WorkflowAggregatedState" = betterproto.enum_field(1) fatal_errors: List["WorkflowFatalError"] = betterproto.message_field(2) execution_id: "ExecutionIdentity" = betterproto.message_field(3) is_recovering: bool = betterproto.bool_field(4) - - -@dataclass(eq=False, repr=False) -class Backpressure(betterproto.Message): - enable_backpressure: bool = betterproto.bool_field(1) - - -@dataclass(eq=False, repr=False) -class CreditUpdate(betterproto.Message): - pass - - -@dataclass(eq=False, repr=False) -class ActorCommand(betterproto.Message): - backpressure: "Backpressure" = betterproto.message_field(1, group="sealed_value") - credit_update: "CreditUpdate" = betterproto.message_field(2, group="sealed_value") - - -@dataclass(eq=False, repr=False) -class PythonActorMessage(betterproto.Message): - payload: "ActorCommand" = betterproto.message_field(1) diff --git a/core/amber/src/main/python/proto/edu/uci/ics/texera/web/__init__.py b/core/amber/src/main/python/proto/edu/uci/ics/texera/web/__init__.py index 8979cc0b8f1..fcbf03a7fad 100644 --- a/core/amber/src/main/python/proto/edu/uci/ics/texera/web/__init__.py +++ b/core/amber/src/main/python/proto/edu/uci/ics/texera/web/__init__.py @@ -1,5 +1,5 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! -# sources: edu/uci/ics/texera/workflowruntimestate.proto +# sources: edu/uci/ics/texera/workflow_runtime_state.proto # plugin: python-betterproto # This file has been @generated diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/clustering/ClusterListener.scala b/core/amber/src/main/scala/edu/uci/ics/amber/clustering/ClusterListener.scala index 9bd461d19a5..63ac4fbab73 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/clustering/ClusterListener.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/clustering/ClusterListener.scala @@ -6,19 +6,16 @@ import akka.cluster.Cluster import com.google.protobuf.timestamp.Timestamp import com.twitter.util.{Await, Future} import edu.uci.ics.amber.clustering.ClusterListener.numWorkerNodesInCluster -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.{ - COMPLETED, - FAILED -} -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import edu.uci.ics.amber.engine.common.{AmberConfig, AmberLogging} import edu.uci.ics.amber.error.ErrorUtils.getStackTraceWithAllCauses import edu.uci.ics.texera.web.SessionState import edu.uci.ics.texera.web.model.websocket.response.ClusterStatusUpdateEvent import edu.uci.ics.texera.web.service.{WorkflowExecutionService, WorkflowService} import edu.uci.ics.texera.web.storage.ExecutionStateStore.updateWorkflowState -import edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.EXECUTION_FAILURE -import edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError +import edu.uci.ics.amber.engine.common.FatalErrorType.EXECUTION_FAILURE +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState._ +import edu.uci.ics.amber.engine.common.WorkflowFatalError import java.time.Instant import scala.collection.mutable.ArrayBuffer @@ -65,7 +62,7 @@ class ClusterListener extends Actor with AmberLogging { private def forcefullyStop(executionService: WorkflowExecutionService, cause: Throwable): Unit = { executionService.client.shutdown() executionService.executionStateStore.statsStore.updateState(stats => - stats.withEndTimeStamp(System.currentTimeMillis()) + stats.withEndTimestamp(System.currentTimeMillis()) ) executionService.executionStateStore.metadataStore.updateState { metadataStore => logger.error("forcefully stopping execution", cause) diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AkkaActorRefMappingService.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AkkaActorRefMappingService.scala index 6d9c1140f28..0a7348063bc 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AkkaActorRefMappingService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AkkaActorRefMappingService.scala @@ -8,8 +8,8 @@ import edu.uci.ics.amber.engine.architecture.common.WorkflowActor.{ RegisterActorRef } import edu.uci.ics.amber.engine.common.{AmberLogging, VirtualIdentityUtils} -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} -import edu.uci.ics.amber.engine.common.virtualidentity.util.{CONTROLLER, SELF} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.util.{CONTROLLER, SELF} import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AkkaActorService.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AkkaActorService.scala index 01367423c58..98a5c173800 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AkkaActorService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AkkaActorService.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.architecture.common import akka.actor.{ActorContext, ActorRef, Address, Cancellable, Props} -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import scala.concurrent.duration.{DurationInt, FiniteDuration} import akka.util.Timeout diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AkkaMessageTransferService.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AkkaMessageTransferService.scala index f89e968620d..bfa7be9bf3a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AkkaMessageTransferService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AkkaMessageTransferService.scala @@ -5,7 +5,7 @@ import edu.uci.ics.amber.engine.architecture.common.WorkflowActor.NetworkMessage import edu.uci.ics.amber.engine.architecture.messaginglayer.{CongestionControl, FlowControl} import edu.uci.ics.amber.engine.common.{AmberConfig, AmberLogging} import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} import scala.collection.mutable import scala.concurrent.duration.DurationInt diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AmberProcessor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AmberProcessor.scala index c7fd40a9d1f..dd68feb16c8 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AmberProcessor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AmberProcessor.scala @@ -5,14 +5,14 @@ import edu.uci.ics.amber.engine.architecture.messaginglayer.{ NetworkInputGateway, NetworkOutputGateway } -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation +import edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation +import edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation import edu.uci.ics.amber.engine.architecture.worker.managers.StatisticsManager import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.MainThreadDelegateMessage import edu.uci.ics.amber.engine.common.AmberLogging import edu.uci.ics.amber.engine.common.ambermessage.{ControlPayload, WorkflowFIFOMessage} import edu.uci.ics.amber.engine.common.rpc.{AsyncRPCClient, AsyncRPCServer} -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} abstract class AmberProcessor( val actorId: ActorVirtualIdentity, diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/ProcessingStepCursor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/ProcessingStepCursor.scala index 7168726de91..9154c425e73 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/ProcessingStepCursor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/ProcessingStepCursor.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.architecture.common import edu.uci.ics.amber.engine.architecture.common.ProcessingStepCursor.INIT_STEP -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity +import edu.uci.ics.amber.engine.common.ChannelIdentity object ProcessingStepCursor { // step value before processing any incoming message diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/WorkflowActor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/WorkflowActor.scala index 103f4916428..f478a892086 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/WorkflowActor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/WorkflowActor.scala @@ -28,7 +28,7 @@ import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.{ import edu.uci.ics.amber.engine.common.{AmberLogging, CheckpointState} import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage import edu.uci.ics.amber.engine.common.storage.SequentialRecordStorage -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} import scala.concurrent.Await import scala.concurrent.duration.DurationInt diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ClientEvent.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ClientEvent.scala index b5e1d3b72ad..44541550456 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ClientEvent.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ClientEvent.scala @@ -1,10 +1,8 @@ package edu.uci.ics.amber.engine.architecture.controller -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessagePayload import edu.uci.ics.amber.engine.common.model.tuple.Tuple -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity -import edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, OperatorMetrics, WorkflowAggregatedState} trait ClientEvent extends WorkflowFIFOMessagePayload diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/Controller.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/Controller.scala index 2811ad42d88..5a43a581854 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/Controller.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/Controller.scala @@ -9,16 +9,16 @@ import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.{ FaultToleranceConfig, StateRestoreConfig } -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ ChannelMarkerPayload, ControlInvocation } import edu.uci.ics.amber.engine.common.ambermessage.WorkflowMessage.getInMemSize import edu.uci.ics.amber.engine.common.model.{PhysicalPlan, WorkflowContext} import edu.uci.ics.amber.engine.common.ambermessage.{ControlPayload, WorkflowFIFOMessage} -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity +import edu.uci.ics.amber.engine.common.ChannelIdentity import edu.uci.ics.amber.engine.common.{AmberConfig, CheckpointState, SerializedState} -import edu.uci.ics.amber.engine.common.virtualidentity.util.{CLIENT, CONTROLLER, SELF} +import edu.uci.ics.amber.engine.common.util.{CLIENT, CONTROLLER, SELF} import edu.uci.ics.texera.workflow.common.storage.OpResultStorage import scala.concurrent.duration.DurationInt diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerAsyncRPCHandlerInitializer.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerAsyncRPCHandlerInitializer.scala index 1285a5a8cdf..d187ef565bb 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerAsyncRPCHandlerInitializer.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerAsyncRPCHandlerInitializer.scala @@ -2,10 +2,10 @@ package edu.uci.ics.amber.engine.architecture.controller import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.promisehandlers._ -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext -import edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceFs2Grpc +import edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext +import edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceFs2Grpc import edu.uci.ics.amber.engine.common.rpc.AsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import edu.uci.ics.amber.engine.common.AmberLogging class ControllerAsyncRPCHandlerInitializer( diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerProcessor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerProcessor.scala index e103add8d78..0041d181a97 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerProcessor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerProcessor.scala @@ -12,7 +12,7 @@ import edu.uci.ics.amber.engine.architecture.scheduling.WorkflowExecutionCoordin import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.MainThreadDelegateMessage import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage import edu.uci.ics.amber.engine.common.model.WorkflowContext -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import edu.uci.ics.texera.workflow.common.storage.OpResultStorage class ControllerProcessor( diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerTimerService.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerTimerService.scala index a3c225529c8..2e64bb2e4f6 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerTimerService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerTimerService.scala @@ -2,13 +2,13 @@ package edu.uci.ics.amber.engine.architecture.controller import akka.actor.Cancellable import edu.uci.ics.amber.engine.architecture.common.AkkaActorService -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, QueryStatisticsRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_CONTROLLER_INITIATE_QUERY_STATISTICS +import edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_CONTROLLER_INITIATE_QUERY_STATISTICS import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient.ControlInvocation -import edu.uci.ics.amber.engine.common.virtualidentity.util.SELF +import edu.uci.ics.amber.engine.common.util.SELF import scala.concurrent.duration.{DurationInt, FiniteDuration, MILLISECONDS} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/GlobalReplayManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/GlobalReplayManager.scala index c332aa2f5b7..4dec2774efc 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/GlobalReplayManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/GlobalReplayManager.scala @@ -1,6 +1,6 @@ package edu.uci.ics.amber.engine.architecture.controller -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/ExecutionUtils.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/ExecutionUtils.scala index 67328bdf419..6277ff4721d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/ExecutionUtils.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/ExecutionUtils.scala @@ -1,8 +1,7 @@ package edu.uci.ics.amber.engine.architecture.controller.execution -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState -import edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping -import edu.uci.ics.amber.engine.common.workflowruntimestate.{OperatorMetrics, OperatorStatistics} +import edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping +import edu.uci.ics.amber.engine.common.{OperatorMetrics, OperatorStatistics, WorkflowAggregatedState} object ExecutionUtils { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/LinkExecution.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/LinkExecution.scala index 41866be37d4..2a5d0ab3f25 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/LinkExecution.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/LinkExecution.scala @@ -1,6 +1,6 @@ package edu.uci.ics.amber.engine.architecture.controller.execution -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity +import edu.uci.ics.amber.engine.common.ChannelIdentity import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/OperatorExecution.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/OperatorExecution.scala index 0612b9458b0..d0b00e7a3f8 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/OperatorExecution.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/OperatorExecution.scala @@ -2,11 +2,8 @@ package edu.uci.ics.amber.engine.architecture.controller.execution import edu.uci.ics.amber.engine.architecture.controller.execution.ExecutionUtils.aggregateStates import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.WorkerExecution -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState -import edu.uci.ics.amber.engine.architecture.worker.statistics.{PortTupleCountMapping, WorkerState} -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity -import edu.uci.ics.amber.engine.common.workflow.PortIdentity -import edu.uci.ics.amber.engine.common.workflowruntimestate.{OperatorMetrics, OperatorStatistics} +import edu.uci.ics.amber.engine.architecture.worker.{PortTupleCountMapping, WorkerState} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, OperatorMetrics, OperatorStatistics, PortIdentity, WorkflowAggregatedState} import java.util import scala.jdk.CollectionConverters._ diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/RegionExecution.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/RegionExecution.scala index 945ae388157..3d5f2791335 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/RegionExecution.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/RegionExecution.scala @@ -1,12 +1,9 @@ package edu.uci.ics.amber.engine.architecture.controller.execution import com.rits.cloning.Cloner -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState import edu.uci.ics.amber.engine.architecture.scheduling.Region -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics -import edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity -import edu.uci.ics.amber.engine.common.workflow.PhysicalLink -import edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics +import edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics +import edu.uci.ics.amber.engine.common.{OperatorMetrics, PhysicalLink, PhysicalOpIdentity, WorkflowAggregatedState} import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/WorkflowExecution.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/WorkflowExecution.scala index ee06da06bd0..c6315334c20 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/WorkflowExecution.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/WorkflowExecution.scala @@ -1,11 +1,9 @@ package edu.uci.ics.amber.engine.architecture.controller.execution import edu.uci.ics.amber.engine.architecture.controller.execution.ExecutionUtils.aggregateMetrics -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState._ import edu.uci.ics.amber.engine.architecture.scheduling.{Region, RegionIdentity} -import edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity -import edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState._ +import edu.uci.ics.amber.engine.common.{OperatorMetrics, PhysicalOpIdentity, WorkflowAggregatedState} import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ChannelMarkerHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ChannelMarkerHandler.scala index 528329963f3..9a1d315ee41 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ChannelMarkerHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ChannelMarkerHandler.scala @@ -2,18 +2,18 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, ControlInvocation, PropagateChannelMarkerRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ ControlReturn, PropagateChannelMarkerResponse } import edu.uci.ics.amber.engine.common.VirtualIdentityUtils -import edu.uci.ics.amber.engine.common.virtualidentity.util.CONTROLLER -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.util.CONTROLLER +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} trait ChannelMarkerHandler { this: ControllerAsyncRPCHandlerInitializer => diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ConsoleMessageHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ConsoleMessageHandler.scala index 6baf61ed3d4..f6155b1ba27 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ConsoleMessageHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ConsoleMessageHandler.scala @@ -2,11 +2,11 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, ConsoleMessageTriggeredRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn trait ConsoleMessageHandler { this: ControllerAsyncRPCHandlerInitializer => diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/DebugCommandHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/DebugCommandHandler.scala index e238417b30e..0a61ab03a4b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/DebugCommandHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/DebugCommandHandler.scala @@ -2,12 +2,12 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, DebugCommandRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity trait DebugCommandHandler { this: ControllerAsyncRPCHandlerInitializer => diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/EvaluatePythonExpressionHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/EvaluatePythonExpressionHandler.scala index e0302eb1688..64bea378d9c 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/EvaluatePythonExpressionHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/EvaluatePythonExpressionHandler.scala @@ -2,12 +2,12 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, EvaluatePythonExpressionRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity +import edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse +import edu.uci.ics.amber.engine.common.OperatorIdentity trait EvaluatePythonExpressionHandler { this: ControllerAsyncRPCHandlerInitializer => diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/LinkWorkersHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/LinkWorkersHandler.scala index e3b74b5a4be..07049ff7f54 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/LinkWorkersHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/LinkWorkersHandler.scala @@ -2,13 +2,7 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ - AddInputChannelRequest, - AddPartitioningRequest, - AsyncRPCContext, - LinkWorkersRequest -} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.{AddInputChannelRequest, AddPartitioningRequest, AsyncRPCContext, EmptyReturn, LinkWorkersRequest} /** add a data transfer partitioning to the sender workers and update input linking * for the receiver workers of a link strategy. diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/PauseHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/PauseHandler.scala index 78bfbb68689..1cddfc86494 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/PauseHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/PauseHandler.scala @@ -2,10 +2,10 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{AsyncRPCContext, EmptyRequest} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.{EmptyReturn, WorkerMetricsResponse} +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, EmptyRequest} +import edu.uci.ics.amber.engine.architecture.rpc.{EmptyReturn, WorkerMetricsResponse} import edu.uci.ics.amber.engine.common.model.tuple.Tuple -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import edu.uci.ics.amber.engine.architecture.controller.{ExecutionStatsUpdate, ExecutionStateUpdate} import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/PortCompletedHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/PortCompletedHandler.scala index 1beb1d805d0..29ed25fb5ad 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/PortCompletedHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/PortCompletedHandler.scala @@ -8,13 +8,13 @@ import edu.uci.ics.amber.engine.architecture.controller.{ import edu.uci.ics.amber.engine.architecture.scheduling.GlobalPortIdentity import edu.uci.ics.amber.engine.common.VirtualIdentityUtils import edu.uci.ics.amber.engine.common.amberexception.WorkflowRuntimeException -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, PortCompletedRequest, QueryStatisticsRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn -import edu.uci.ics.amber.engine.common.virtualidentity.util.CONTROLLER +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn +import edu.uci.ics.amber.engine.common.util.CONTROLLER /** Notify the completion of a port: * - For input port, it means the worker has finished consuming and processing all the data diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/QueryWorkerStatisticsHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/QueryWorkerStatisticsHandler.scala index 9f10da7a14e..04ba11f6819 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/QueryWorkerStatisticsHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/QueryWorkerStatisticsHandler.scala @@ -5,12 +5,12 @@ import edu.uci.ics.amber.engine.architecture.controller.{ ControllerAsyncRPCHandlerInitializer, ExecutionStatsUpdate } -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, EmptyRequest, QueryStatisticsRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.common.VirtualIdentityUtils /** Get statistics from all the workers diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ResumeHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ResumeHandler.scala index 9488467e4f3..96bbc0e1d59 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ResumeHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ResumeHandler.scala @@ -2,8 +2,8 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{AsyncRPCContext, EmptyRequest} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, EmptyRequest} +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.common.VirtualIdentityUtils import edu.uci.ics.amber.engine.architecture.controller.ExecutionStatsUpdate diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/RetrieveWorkflowStateHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/RetrieveWorkflowStateHandler.scala index ab8817a3321..62456a2d41a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/RetrieveWorkflowStateHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/RetrieveWorkflowStateHandler.scala @@ -2,19 +2,19 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.NO_ALIGNMENT -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.NO_ALIGNMENT +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, EmptyRequest, PropagateChannelMarkerRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ RetrieveWorkflowStateResponse, StringResponse } -import edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_RETRIEVE_STATE -import edu.uci.ics.amber.engine.common.virtualidentity.util.SELF -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity +import edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_RETRIEVE_STATE +import edu.uci.ics.amber.engine.common.util.SELF +import edu.uci.ics.amber.engine.common.ChannelMarkerIdentity import java.time.Instant trait RetrieveWorkflowStateHandler { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/RetryWorkflowHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/RetryWorkflowHandler.scala index 60f2f3d793c..f21c92c3b57 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/RetryWorkflowHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/RetryWorkflowHandler.scala @@ -2,13 +2,13 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, EmptyRequest, RetryWorkflowRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn -import edu.uci.ics.amber.engine.common.virtualidentity.util.CONTROLLER +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn +import edu.uci.ics.amber.engine.common.util.CONTROLLER /** retry the execution of the entire workflow * diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/StartWorkflowHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/StartWorkflowHandler.scala index 972a00a2b8e..a2fa031c3d7 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/StartWorkflowHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/StartWorkflowHandler.scala @@ -2,9 +2,9 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{AsyncRPCContext, EmptyRequest} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.RUNNING -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, EmptyRequest} +import edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState.RUNNING /** start the workflow by starting the source workers * note that this SHOULD only be called once per workflow diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/TakeGlobalCheckpointHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/TakeGlobalCheckpointHandler.scala index 6aac68fa6ce..20d6befd713 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/TakeGlobalCheckpointHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/TakeGlobalCheckpointHandler.scala @@ -2,20 +2,20 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.NO_ALIGNMENT -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.NO_ALIGNMENT +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, FinalizeCheckpointRequest, PrepareCheckpointRequest, PropagateChannelMarkerRequest, TakeGlobalCheckpointRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse -import edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_PREPARE_CHECKPOINT +import edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse +import edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_PREPARE_CHECKPOINT import edu.uci.ics.amber.engine.common.{CheckpointState, SerializedState} import edu.uci.ics.amber.engine.common.storage.SequentialRecordStorage -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity -import edu.uci.ics.amber.engine.common.virtualidentity.util.SELF +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.util.SELF import java.net.URI diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerExecutionCompletedHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerExecutionCompletedHandler.scala index a9eca0e3975..dfda496ff15 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerExecutionCompletedHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerExecutionCompletedHandler.scala @@ -5,13 +5,13 @@ import edu.uci.ics.amber.engine.architecture.controller.{ ControllerAsyncRPCHandlerInitializer, ExecutionStateUpdate } -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, EmptyRequest, QueryStatisticsRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn -import edu.uci.ics.amber.engine.common.virtualidentity.util.SELF +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn +import edu.uci.ics.amber.engine.common.util.SELF /** indicate a worker has completed its execution * i.e. received and processed all data from upstreams diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerStateUpdatedHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerStateUpdatedHandler.scala index b8f34bb638d..206c88f5b42 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerStateUpdatedHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerStateUpdatedHandler.scala @@ -5,11 +5,11 @@ import edu.uci.ics.amber.engine.architecture.controller.{ ControllerAsyncRPCHandlerInitializer, ExecutionStatsUpdate } -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, WorkerStateUpdatedRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.common.VirtualIdentityUtils /** indicate the state change of a worker diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/deploysemantics/layer/WorkerExecution.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/deploysemantics/layer/WorkerExecution.scala index 387898bd982..e8159954e71 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/deploysemantics/layer/WorkerExecution.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/deploysemantics/layer/WorkerExecution.scala @@ -1,9 +1,9 @@ package edu.uci.ics.amber.engine.architecture.deploysemantics.layer import edu.uci.ics.amber.engine.architecture.controller.execution.WorkerPortExecution -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.UNINITIALIZED -import edu.uci.ics.amber.engine.architecture.worker.statistics.{WorkerState, WorkerStatistics} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.architecture.worker.WorkerState.UNINITIALIZED +import edu.uci.ics.amber.engine.architecture.worker.{WorkerState, WorkerStatistics} +import edu.uci.ics.amber.engine.common.PortIdentity import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/EmptyReplayLogger.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/EmptyReplayLogger.scala index 4b0d760c656..24a1ac4ac5d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/EmptyReplayLogger.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/EmptyReplayLogger.scala @@ -1,6 +1,6 @@ package edu.uci.ics.amber.engine.architecture.logreplay import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage -import edu.uci.ics.amber.engine.common.virtualidentity.{ChannelIdentity, ChannelMarkerIdentity} +import edu.uci.ics.amber.engine.common.{ChannelIdentity, ChannelMarkerIdentity} class EmptyReplayLogger extends ReplayLogger { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/OrderEnforcer.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/OrderEnforcer.scala index 48077de232a..16ab130a216 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/OrderEnforcer.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/OrderEnforcer.scala @@ -1,6 +1,6 @@ package edu.uci.ics.amber.engine.architecture.logreplay -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity +import edu.uci.ics.amber.engine.common.ChannelIdentity trait OrderEnforcer { var isCompleted: Boolean diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLogGenerator.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLogGenerator.scala index 706db1224ac..e6797e3bb2e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLogGenerator.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLogGenerator.scala @@ -2,7 +2,7 @@ package edu.uci.ics.amber.engine.architecture.logreplay import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage import edu.uci.ics.amber.engine.common.storage.SequentialRecordStorage -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity +import edu.uci.ics.amber.engine.common.ChannelMarkerIdentity import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLogManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLogManager.scala index edb09c85180..a82d1d1af99 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLogManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLogManager.scala @@ -5,7 +5,7 @@ import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.MainThreadDel import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage import edu.uci.ics.amber.engine.common.storage.SequentialRecordStorage.SequentialRecordWriter import edu.uci.ics.amber.engine.common.storage.{EmptyRecordStorage, SequentialRecordStorage} -import edu.uci.ics.amber.engine.common.virtualidentity.{ChannelIdentity, ChannelMarkerIdentity} +import edu.uci.ics.amber.engine.common.{ChannelIdentity, ChannelMarkerIdentity} //In-mem formats: sealed trait ReplayLogRecord diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLogger.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLogger.scala index da758cca498..1809af4202b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLogger.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLogger.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.architecture.logreplay import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage -import edu.uci.ics.amber.engine.common.virtualidentity.{ChannelIdentity, ChannelMarkerIdentity} +import edu.uci.ics.amber.engine.common.{ChannelIdentity, ChannelMarkerIdentity} abstract class ReplayLogger { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLoggerImpl.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLoggerImpl.scala index 86bf4c2a6f2..613c8d710c7 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLoggerImpl.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayLoggerImpl.scala @@ -2,7 +2,7 @@ package edu.uci.ics.amber.engine.architecture.logreplay import edu.uci.ics.amber.engine.architecture.common.ProcessingStepCursor.INIT_STEP import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage -import edu.uci.ics.amber.engine.common.virtualidentity.{ChannelIdentity, ChannelMarkerIdentity} +import edu.uci.ics.amber.engine.common.{ChannelIdentity, ChannelMarkerIdentity} import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayOrderEnforcer.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayOrderEnforcer.scala index 86fb3d07049..2be6514243e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayOrderEnforcer.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/logreplay/ReplayOrderEnforcer.scala @@ -1,6 +1,6 @@ package edu.uci.ics.amber.engine.architecture.logreplay -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity +import edu.uci.ics.amber.engine.common.ChannelIdentity import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/AmberFIFOChannel.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/AmberFIFOChannel.scala index d37cd7dabe3..5c5048ddf5e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/AmberFIFOChannel.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/AmberFIFOChannel.scala @@ -3,8 +3,8 @@ package edu.uci.ics.amber.engine.architecture.messaginglayer import edu.uci.ics.amber.engine.common.AmberLogging import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage import edu.uci.ics.amber.engine.common.ambermessage.WorkflowMessage.getInMemSize -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.PortIdentity import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/InputGateway.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/InputGateway.scala index da7ece8b16b..a73a132a775 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/InputGateway.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/InputGateway.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.architecture.messaginglayer import edu.uci.ics.amber.engine.architecture.logreplay.OrderEnforcer -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity +import edu.uci.ics.amber.engine.common.ChannelIdentity trait InputGateway { def tryPickControlChannel: Option[AmberFIFOChannel] diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/InputManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/InputManager.scala index adfdf2812a8..0d3164e59c3 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/InputManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/InputManager.scala @@ -2,8 +2,8 @@ package edu.uci.ics.amber.engine.architecture.messaginglayer import edu.uci.ics.amber.engine.common.AmberLogging import edu.uci.ics.amber.engine.common.model.tuple.{Schema, Tuple} -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.PortIdentity import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/NetworkInputGateway.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/NetworkInputGateway.scala index fbf239a9400..963c7eeff3c 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/NetworkInputGateway.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/NetworkInputGateway.scala @@ -2,7 +2,7 @@ package edu.uci.ics.amber.engine.architecture.messaginglayer import edu.uci.ics.amber.engine.architecture.logreplay.OrderEnforcer import edu.uci.ics.amber.engine.common.AmberLogging -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/NetworkOutputGateway.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/NetworkOutputGateway.scala index 23204596f67..1b0f1c8da9e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/NetworkOutputGateway.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/NetworkOutputGateway.scala @@ -9,8 +9,8 @@ import edu.uci.ics.amber.engine.common.ambermessage.{ } import java.util.concurrent.atomic.AtomicLong -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} -import edu.uci.ics.amber.engine.common.virtualidentity.util.SELF +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.util.SELF import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/OutputManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/OutputManager.scala index 0239dfaa6ae..66dc27ee4ad 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/OutputManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/OutputManager.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.architecture.messaginglayer.OutputManager.{ toPartitioner } import edu.uci.ics.amber.engine.architecture.sendsemantics.partitioners._ -import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings._ +import edu.uci.ics.amber.engine.architecture.sendsemantics._ import edu.uci.ics.amber.engine.common.AmberLogging import edu.uci.ics.amber.engine.common.model.Marker import edu.uci.ics.amber.engine.common.model.tuple.{ @@ -16,8 +16,8 @@ import edu.uci.ics.amber.engine.common.model.tuple.{ SchemaEnforceable, TupleLike } -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} -import edu.uci.ics.amber.engine.common.workflow.{PhysicalLink, PortIdentity} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.{PhysicalLink, PortIdentity} import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/WorkerPort.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/WorkerPort.scala index 0cd4e20fcf7..aec8d6c6367 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/WorkerPort.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/WorkerPort.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.architecture.messaginglayer import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity +import edu.uci.ics.amber.engine.common.ChannelIdentity import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/WorkerTimerService.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/WorkerTimerService.scala index 6fb1cb54c5e..e2b4de356b2 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/WorkerTimerService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/WorkerTimerService.scala @@ -2,12 +2,12 @@ package edu.uci.ics.amber.engine.architecture.messaginglayer import akka.actor.Cancellable import edu.uci.ics.amber.engine.architecture.common.AkkaActorService -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{AsyncRPCContext, EmptyRequest} -import edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_FLUSH_NETWORK_BUFFER +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, EmptyRequest} +import edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_FLUSH_NETWORK_BUFFER import edu.uci.ics.amber.engine.common.AmberConfig import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient.ControlInvocation -import edu.uci.ics.amber.engine.common.virtualidentity.util.SELF +import edu.uci.ics.amber.engine.common.util.SELF import scala.concurrent.duration.{DurationInt, FiniteDuration, MILLISECONDS} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyClient.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyClient.scala index 4fc056c3bcf..ce5cbd4c55e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyClient.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyClient.scala @@ -3,35 +3,18 @@ package edu.uci.ics.amber.engine.architecture.pythonworker import com.google.protobuf.ByteString import com.google.protobuf.any.Any import com.twitter.util.{Await, Promise} -import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.{ - OpExecInitInfo, - OpExecInitInfoWithCode -} -import edu.uci.ics.amber.engine.architecture.pythonworker.WorkerBatchInternalQueue.{ - ActorCommandElement, - ControlElement, - DataElement -} -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ - ControlInvocation, - InitializeExecutorRequest -} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation +import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.{OpExecInitInfo, OpExecInitInfoWithCode} +import edu.uci.ics.amber.engine.architecture.python._ +import edu.uci.ics.amber.engine.architecture.pythonworker.WorkerBatchInternalQueue.{ActorCommandElement, ControlElement, DataElement} +import edu.uci.ics.amber.engine.architecture.rpc.{ControlInvocation, InitializeExecutorRequest} +import edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation import edu.uci.ics.amber.engine.common.{AmberLogging, AmberRuntime} -import edu.uci.ics.amber.engine.common.actormessage.{ActorCommand, PythonActorMessage} +import edu.uci.ics.amber.engine.common.ActorCommand import edu.uci.ics.amber.engine.common.amberexception.WorkflowRuntimeException -import edu.uci.ics.amber.engine.common.ambermessage.{ - ControlPayload, - ControlPayloadV2, - DataFrame, - DataPayload, - MarkerFrame, - PythonControlMessage, - PythonDataHeader -} +import edu.uci.ics.amber.engine.common.ambermessage.{ControlPayload, DataFrame, DataPayload, MarkerFrame} import edu.uci.ics.amber.engine.common.model.State import edu.uci.ics.amber.engine.common.model.tuple.{Schema, Tuple} -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import org.apache.arrow.flight._ import org.apache.arrow.memory.{ArrowBuf, BufferAllocator, RootAllocator} import org.apache.arrow.vector.VectorSchemaRoot diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyServer.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyServer.scala index 5000c5b0f09..0085d69806f 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyServer.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyServer.scala @@ -2,9 +2,8 @@ package edu.uci.ics.amber.engine.architecture.pythonworker import com.google.common.primitives.Longs import edu.uci.ics.amber.engine.architecture.messaginglayer.NetworkOutputGateway -import edu.uci.ics.amber.engine.common.AmberLogging +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, AmberLogging} import edu.uci.ics.amber.engine.common.ambermessage._ -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity import org.apache.arrow.flight._ import org.apache.arrow.memory.{ArrowBuf, BufferAllocator, RootAllocator} import org.apache.arrow.util.AutoCloseables @@ -15,10 +14,8 @@ import java.net.ServerSocket import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable import com.twitter.util.Promise -import edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.{ - ControlInvocation => ControlInvocationV2, - ReturnInvocation => ReturnInvocationV2 -} +import edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.{ControlInvocation => ControlInvocationV2, ReturnInvocation => ReturnInvocationV2} +import edu.uci.ics.amber.engine.architecture.python.{PythonControlMessage, PythonDataHeader} import edu.uci.ics.amber.engine.common.model.{EndOfInputChannel, StartOfInputChannel, State} import edu.uci.ics.amber.engine.common.model.tuple.Tuple diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonWorkflowWorker.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonWorkflowWorker.scala index a77ebc00908..00d083931bd 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonWorkflowWorker.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonWorkflowWorker.scala @@ -12,10 +12,10 @@ import edu.uci.ics.amber.engine.architecture.messaginglayer.{ import edu.uci.ics.amber.engine.architecture.pythonworker.WorkerBatchInternalQueue.DataElement import edu.uci.ics.amber.engine.architecture.scheduling.config.WorkerConfig import edu.uci.ics.amber.engine.common.{CheckpointState, Utils} -import edu.uci.ics.amber.engine.common.actormessage.{Backpressure, CreditUpdate} +import edu.uci.ics.amber.engine.common.{Backpressure, CreditUpdate} import edu.uci.ics.amber.engine.common.ambermessage.WorkflowMessage.getInMemSize import edu.uci.ics.amber.engine.common.ambermessage._ -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity +import edu.uci.ics.amber.engine.common.ChannelIdentity import java.nio.file.Path import java.util.concurrent.{ExecutorService, Executors} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/WorkerBatchInternalQueue.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/WorkerBatchInternalQueue.scala index 5532402fdcf..14da0e36af3 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/WorkerBatchInternalQueue.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/WorkerBatchInternalQueue.scala @@ -1,14 +1,14 @@ package edu.uci.ics.amber.engine.architecture.pythonworker +import edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2 import edu.uci.ics.amber.engine.architecture.pythonworker.WorkerBatchInternalQueue._ -import edu.uci.ics.amber.engine.common.actormessage.ActorCommand +import edu.uci.ics.amber.engine.common.{ActorCommand} import edu.uci.ics.amber.engine.common.ambermessage.{ ControlPayload, - ControlPayloadV2, DataFrame, DataPayload } -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity +import edu.uci.ics.amber.engine.common.ChannelIdentity import lbmq.LinkedBlockingMultiQueue import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/CostBasedRegionPlanGenerator.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/CostBasedRegionPlanGenerator.scala index 48e8c85dd9f..5bc61cc13c0 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/CostBasedRegionPlanGenerator.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/CostBasedRegionPlanGenerator.scala @@ -3,8 +3,8 @@ package edu.uci.ics.amber.engine.architecture.scheduling import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.common.AmberConfig import edu.uci.ics.amber.engine.common.model.{PhysicalPlan, WorkflowContext} -import edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity -import edu.uci.ics.amber.engine.common.workflow.PhysicalLink +import edu.uci.ics.amber.engine.common.PhysicalOpIdentity +import edu.uci.ics.amber.engine.common.PhysicalLink import edu.uci.ics.texera.workflow.common.storage.OpResultStorage import org.jgrapht.alg.connectivity.BiconnectivityInspector import org.jgrapht.graph.{DirectedAcyclicGraph, DirectedPseudograph} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/ExpansionGreedyRegionPlanGenerator.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/ExpansionGreedyRegionPlanGenerator.scala index b08be7a8f9a..58a5f41a88a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/ExpansionGreedyRegionPlanGenerator.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/ExpansionGreedyRegionPlanGenerator.scala @@ -3,8 +3,8 @@ package edu.uci.ics.amber.engine.architecture.scheduling import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.common.amberexception.WorkflowRuntimeException import edu.uci.ics.amber.engine.common.model.{PhysicalPlan, WorkflowContext} -import edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity -import edu.uci.ics.amber.engine.common.workflow.PhysicalLink +import edu.uci.ics.amber.engine.common.PhysicalOpIdentity +import edu.uci.ics.amber.engine.common.PhysicalLink import edu.uci.ics.texera.workflow.common.storage.OpResultStorage import org.jgrapht.alg.connectivity.BiconnectivityInspector import org.jgrapht.graph.DirectedAcyclicGraph diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/Region.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/Region.scala index 924952e813a..a76721a8158 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/Region.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/Region.scala @@ -2,8 +2,8 @@ package edu.uci.ics.amber.engine.architecture.scheduling import edu.uci.ics.amber.engine.architecture.scheduling.config.ResourceConfig import edu.uci.ics.amber.engine.common.model.PhysicalOp -import edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity -import edu.uci.ics.amber.engine.common.workflow.{PhysicalLink, PortIdentity} +import edu.uci.ics.amber.engine.common.PhysicalOpIdentity +import edu.uci.ics.amber.engine.common.{PhysicalLink, PortIdentity} import org.jgrapht.graph.{DefaultEdge, DirectedAcyclicGraph} import org.jgrapht.traverse.TopologicalOrderIterator diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala index 4a92bb411dd..914bbf23839 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala @@ -4,31 +4,15 @@ import com.google.protobuf.any.Any import com.google.protobuf.ByteString import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.common.AkkaActorService -import edu.uci.ics.amber.engine.architecture.controller.{ - ControllerConfig, - ExecutionStatsUpdate, - WorkerAssignmentUpdate -} -import edu.uci.ics.amber.engine.architecture.controller.execution.{ - OperatorExecution, - WorkflowExecution -} -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ - AssignPortRequest, - EmptyRequest, - InitializeExecutorRequest, - LinkWorkersRequest -} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.{ - EmptyReturn, - WorkflowAggregatedState -} +import edu.uci.ics.amber.engine.architecture.controller.{ControllerConfig, ExecutionStatsUpdate, WorkerAssignmentUpdate} +import edu.uci.ics.amber.engine.architecture.controller.execution.{OperatorExecution, WorkflowExecution} +import edu.uci.ics.amber.engine.architecture.rpc.{AssignPortRequest, EmptyRequest, InitializeExecutorRequest, LinkWorkersRequest} +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.scheduling.config.{OperatorConfig, ResourceConfig} -import edu.uci.ics.amber.engine.common.AmberRuntime +import edu.uci.ics.amber.engine.common.{AmberRuntime, PhysicalLink, WorkflowAggregatedState} import edu.uci.ics.amber.engine.common.model.PhysicalOp import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient -import edu.uci.ics.amber.engine.common.virtualidentity.util.CONTROLLER -import edu.uci.ics.amber.engine.common.workflow.PhysicalLink +import edu.uci.ics.amber.engine.common.util.CONTROLLER class RegionExecutionCoordinator( region: Region, diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionPlan.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionPlan.scala index d588309c7f6..dda05b2a820 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionPlan.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionPlan.scala @@ -1,5 +1,5 @@ package edu.uci.ics.amber.engine.architecture.scheduling -import edu.uci.ics.amber.engine.common.workflow.PhysicalLink +import edu.uci.ics.amber.engine.common.PhysicalLink import org.jgrapht.graph.DirectedAcyclicGraph import org.jgrapht.traverse.TopologicalOrderIterator diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionPlanGenerator.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionPlanGenerator.scala index 25c0dc44545..baea9fa53e0 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionPlanGenerator.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionPlanGenerator.scala @@ -6,8 +6,8 @@ import edu.uci.ics.amber.engine.architecture.scheduling.resourcePolicies.{ ExecutionClusterInfo } import edu.uci.ics.amber.engine.common.model.{PhysicalOp, PhysicalPlan, WorkflowContext} -import edu.uci.ics.amber.engine.common.virtualidentity.{OperatorIdentity, PhysicalOpIdentity} -import edu.uci.ics.amber.engine.common.workflow.PhysicalLink +import edu.uci.ics.amber.engine.common.{OperatorIdentity, PhysicalOpIdentity} +import edu.uci.ics.amber.engine.common.PhysicalLink import edu.uci.ics.texera.workflow.common.storage.OpResultStorage import edu.uci.ics.texera.workflow.operators.sink.managed.ProgressiveSinkOpDesc import edu.uci.ics.texera.workflow.operators.source.cache.CacheSourceOpDesc diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/WorkflowExecutionCoordinator.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/WorkflowExecutionCoordinator.scala index 7f76932cfd0..025675ae608 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/WorkflowExecutionCoordinator.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/WorkflowExecutionCoordinator.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.architecture.common.AkkaActorService import edu.uci.ics.amber.engine.architecture.controller.ControllerConfig import edu.uci.ics.amber.engine.architecture.controller.execution.WorkflowExecution import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient -import edu.uci.ics.amber.engine.common.workflow.PhysicalLink +import edu.uci.ics.amber.engine.common.PhysicalLink import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/ChannelConfig.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/ChannelConfig.scala index 3c03e928603..6516debf21d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/ChannelConfig.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/ChannelConfig.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.architecture.scheduling.config -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.PortIdentity import edu.uci.ics.texera.workflow.common.workflow.{ BroadcastPartition, HashPartition, diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/LinkConfig.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/LinkConfig.scala index e05b50becef..bb538539f91 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/LinkConfig.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/LinkConfig.scala @@ -1,5 +1,5 @@ package edu.uci.ics.amber.engine.architecture.scheduling.config -import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.{ +import edu.uci.ics.amber.engine.architecture.sendsemantics.{ BroadcastPartitioning, HashBasedShufflePartitioning, OneToOnePartitioning, @@ -7,7 +7,7 @@ import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.{ RangeBasedShufflePartitioning, RoundRobinPartitioning } -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} import edu.uci.ics.texera.workflow.common.workflow.{ BroadcastPartition, HashPartition, diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/ResourceConfig.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/ResourceConfig.scala index 7fb6ef9f1a6..4cd23b4f431 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/ResourceConfig.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/ResourceConfig.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.architecture.scheduling.config -import edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity -import edu.uci.ics.amber.engine.common.workflow.PhysicalLink +import edu.uci.ics.amber.engine.common.PhysicalOpIdentity +import edu.uci.ics.amber.engine.common.PhysicalLink case class ResourceConfig( operatorConfigs: Map[PhysicalOpIdentity, OperatorConfig], diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/WorkerConfig.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/WorkerConfig.scala index f6be9264f45..c4596b39847 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/WorkerConfig.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/config/WorkerConfig.scala @@ -2,7 +2,7 @@ package edu.uci.ics.amber.engine.architecture.scheduling.config import edu.uci.ics.amber.engine.common.model.PhysicalOp import edu.uci.ics.amber.engine.common.{AmberConfig, VirtualIdentityUtils} -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity case object WorkerConfig { def generateWorkerConfigs(physicalOp: PhysicalOp): List[WorkerConfig] = { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/resourcePolicies/ResourceAllocator.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/resourcePolicies/ResourceAllocator.scala index 966ebf5df66..eda075fbcda 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/resourcePolicies/ResourceAllocator.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/resourcePolicies/ResourceAllocator.scala @@ -10,8 +10,8 @@ import edu.uci.ics.amber.engine.architecture.scheduling.config.{ ResourceConfig } import edu.uci.ics.amber.engine.common.model.PhysicalPlan -import edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity -import edu.uci.ics.amber.engine.common.workflow.{PhysicalLink, PortIdentity} +import edu.uci.ics.amber.engine.common.PhysicalOpIdentity +import edu.uci.ics.amber.engine.common.{PhysicalLink, PortIdentity} import edu.uci.ics.texera.workflow.common.workflow.{PartitionInfo, UnknownPartition} import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/BroadcastPartitioner.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/BroadcastPartitioner.scala index e155b64f8e4..300cf79b0cb 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/BroadcastPartitioner.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/BroadcastPartitioner.scala @@ -1,8 +1,8 @@ package edu.uci.ics.amber.engine.architecture.sendsemantics.partitioners -import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning +import edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning import edu.uci.ics.amber.engine.common.model.tuple.Tuple -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity case class BroadcastPartitioner(partitioning: BroadcastPartitioning) extends Partitioner { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/HashBasedShufflePartitioner.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/HashBasedShufflePartitioner.scala index 8aebe1c1349..20b18b7b987 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/HashBasedShufflePartitioner.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/HashBasedShufflePartitioner.scala @@ -1,8 +1,8 @@ package edu.uci.ics.amber.engine.architecture.sendsemantics.partitioners -import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning +import edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning import edu.uci.ics.amber.engine.common.model.tuple.Tuple -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity case class HashBasedShufflePartitioner(partitioning: HashBasedShufflePartitioning) extends Partitioner { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/OneToOnePartitioner.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/OneToOnePartitioner.scala index 01f4f0d1250..51014d8721b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/OneToOnePartitioner.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/OneToOnePartitioner.scala @@ -1,8 +1,8 @@ package edu.uci.ics.amber.engine.architecture.sendsemantics.partitioners -import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning +import edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning import edu.uci.ics.amber.engine.common.model.tuple.Tuple -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity case class OneToOnePartitioner(partitioning: OneToOnePartitioning, actorId: ActorVirtualIdentity) extends Partitioner { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/Partitioner.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/Partitioner.scala index a43008f1b9a..8ad7b55d058 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/Partitioner.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/Partitioner.scala @@ -5,7 +5,7 @@ import edu.uci.ics.amber.engine.common.AmberConfig import edu.uci.ics.amber.engine.common.ambermessage.{DataFrame, MarkerFrame} import edu.uci.ics.amber.engine.common.model.Marker import edu.uci.ics.amber.engine.common.model.tuple.Tuple -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import scala.collection.mutable.ArrayBuffer diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/RangeBasedShufflePartitioner.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/RangeBasedShufflePartitioner.scala index d961635c94a..000aac64b71 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/RangeBasedShufflePartitioner.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/RangeBasedShufflePartitioner.scala @@ -1,8 +1,8 @@ package edu.uci.ics.amber.engine.architecture.sendsemantics.partitioners -import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning +import edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Tuple} -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity case class RangeBasedShufflePartitioner(partitioning: RangeBasedShufflePartitioning) extends Partitioner { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/RoundRobinPartitioner.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/RoundRobinPartitioner.scala index 5bf3bc5bfa5..06cf93b160c 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/RoundRobinPartitioner.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/sendsemantics/partitioners/RoundRobinPartitioner.scala @@ -1,8 +1,8 @@ package edu.uci.ics.amber.engine.architecture.sendsemantics.partitioners -import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning +import edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning import edu.uci.ics.amber.engine.common.model.tuple.Tuple -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity case class RoundRobinPartitioner(partitioning: RoundRobinPartitioning) extends Partitioner { private var roundRobinIndex = 0 diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/ChannelMarkerManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/ChannelMarkerManager.scala index 2f74317dd4a..7e7005c42a4 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/ChannelMarkerManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/ChannelMarkerManager.scala @@ -1,13 +1,13 @@ package edu.uci.ics.amber.engine.architecture.worker import edu.uci.ics.amber.engine.architecture.messaginglayer.InputGateway -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.{ +import edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload +import edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.{ NO_ALIGNMENT, REQUIRE_ALIGNMENT } import edu.uci.ics.amber.engine.common.{AmberLogging, CheckpointState} -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.common.{ ActorVirtualIdentity, ChannelIdentity, ChannelMarkerIdentity diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DPThread.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DPThread.scala index 0d3050c004d..ad56de26855 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DPThread.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DPThread.scala @@ -5,17 +5,17 @@ import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.{ MainThreadDelegateMessage } import edu.uci.ics.amber.engine.architecture.logreplay.ReplayLogManager -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.{READY, UNINITIALIZED} +import edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload +import edu.uci.ics.amber.engine.architecture.worker.WorkerState.{READY, UNINITIALIZED} import edu.uci.ics.amber.engine.common.AmberLogging -import edu.uci.ics.amber.engine.common.actormessage.{ActorCommand, Backpressure} +import edu.uci.ics.amber.engine.common.{ActorCommand, Backpressure} import edu.uci.ics.amber.engine.common.ambermessage.{ ControlPayload, DataPayload, WorkflowFIFOMessage } -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} -import edu.uci.ics.amber.engine.common.virtualidentity.util.SELF +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.util.SELF import edu.uci.ics.amber.error.ErrorUtils.safely import java.util.concurrent.{ diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessor.scala index ef0bd4632bf..78877be7f92 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessor.scala @@ -8,8 +8,8 @@ import edu.uci.ics.amber.engine.architecture.messaginglayer.{ OutputManager, WorkerTimerService } -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.REQUIRE_ALIGNMENT -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.REQUIRE_ALIGNMENT +import edu.uci.ics.amber.engine.architecture.rpc.{ ChannelMarkerPayload, ConsoleMessageTriggeredRequest, EmptyRequest, @@ -18,12 +18,12 @@ import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ } import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.MainThreadDelegateMessage import edu.uci.ics.amber.engine.architecture.worker.managers.SerializationManager -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.{ +import edu.uci.ics.amber.engine.architecture.worker.WorkerState.{ COMPLETED, READY, RUNNING } -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics +import edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics import edu.uci.ics.amber.engine.common.ambermessage._ import edu.uci.ics.amber.engine.common.executor.OperatorExecutor import edu.uci.ics.amber.engine.common.model.{EndOfInputChannel, StartOfInputChannel, State} @@ -35,9 +35,9 @@ import edu.uci.ics.amber.engine.common.model.tuple.{ TupleLike } import edu.uci.ics.amber.engine.common.statetransition.WorkerStateManager -import edu.uci.ics.amber.engine.common.virtualidentity.util.CONTROLLER -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.util.CONTROLLER +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.PortIdentity import edu.uci.ics.amber.error.ErrorUtils.{mkConsoleMessage, safely} class DataProcessor( diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessorRPCHandlerInitializer.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessorRPCHandlerInitializer.scala index 0c36fc55fa3..8f282968331 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessorRPCHandlerInitializer.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessorRPCHandlerInitializer.scala @@ -1,17 +1,17 @@ package edu.uci.ics.amber.engine.architecture.worker import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, DebugCommandRequest, EmptyRequest, EvaluatePythonExpressionRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.{EmptyReturn, EvaluatedValue} -import edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceFs2Grpc +import edu.uci.ics.amber.engine.architecture.rpc.{EmptyReturn, EvaluatedValue} +import edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceFs2Grpc import edu.uci.ics.amber.engine.architecture.worker.promisehandlers._ import edu.uci.ics.amber.engine.common.rpc.AsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import edu.uci.ics.amber.engine.common.AmberLogging class DataProcessorRPCHandlerInitializer(val dp: DataProcessor) diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/PauseManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/PauseManager.scala index 5b45d23e05d..e870026b55c 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/PauseManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/PauseManager.scala @@ -2,7 +2,7 @@ package edu.uci.ics.amber.engine.architecture.worker import edu.uci.ics.amber.engine.architecture.messaginglayer.InputGateway import edu.uci.ics.amber.engine.common.AmberLogging -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/PauseType.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/PauseType.scala index 61396c19c04..216965a340e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/PauseType.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/PauseType.scala @@ -1,6 +1,6 @@ package edu.uci.ics.amber.engine.architecture.worker -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity +import edu.uci.ics.amber.engine.common.ChannelMarkerIdentity sealed trait PauseType diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/WorkflowWorker.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/WorkflowWorker.scala index 37230ac3e47..5661609bc98 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/WorkflowWorker.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/WorkflowWorker.scala @@ -5,14 +5,14 @@ import edu.uci.ics.amber.engine.architecture.common.WorkflowActor import edu.uci.ics.amber.engine.architecture.common.WorkflowActor.NetworkAck import edu.uci.ics.amber.engine.architecture.controller.ReplayStatusUpdate import edu.uci.ics.amber.engine.architecture.messaginglayer.WorkerTimerService -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation +import edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation import edu.uci.ics.amber.engine.architecture.scheduling.config.WorkerConfig import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker._ import edu.uci.ics.amber.engine.common.{CheckpointState, SerializedState} -import edu.uci.ics.amber.engine.common.actormessage.{ActorCommand, Backpressure} +import edu.uci.ics.amber.engine.common.{ActorCommand, Backpressure} import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage import edu.uci.ics.amber.engine.common.ambermessage.WorkflowMessage.getInMemSize -import edu.uci.ics.amber.engine.common.virtualidentity.{ChannelIdentity, ChannelMarkerIdentity} +import edu.uci.ics.amber.engine.common.{ChannelIdentity, ChannelMarkerIdentity} import java.net.URI import java.util.concurrent.LinkedBlockingQueue diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/SerializationManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/SerializationManager.scala index 4d23dd7440b..0ffceb0406c 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/SerializationManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/SerializationManager.scala @@ -2,7 +2,7 @@ package edu.uci.ics.amber.engine.architecture.worker.managers import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo.generateJavaOpExec -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest +import edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest import edu.uci.ics.amber.engine.common.executor.OperatorExecutor import edu.uci.ics.amber.engine.common.model.tuple.TupleLike import edu.uci.ics.amber.engine.common.{ @@ -12,8 +12,8 @@ import edu.uci.ics.amber.engine.common.{ CheckpointSupport, VirtualIdentityUtils } -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.PortIdentity class SerializationManager(val actorId: ActorVirtualIdentity) extends AmberLogging { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/StatisticsManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/StatisticsManager.scala index 9f47679595d..729e5da35b6 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/StatisticsManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/StatisticsManager.scala @@ -1,11 +1,11 @@ package edu.uci.ics.amber.engine.architecture.worker.managers -import edu.uci.ics.amber.engine.architecture.worker.statistics.{ +import edu.uci.ics.amber.engine.architecture.worker.{ PortTupleCountMapping, WorkerStatistics } import edu.uci.ics.amber.engine.common.executor.{OperatorExecutor, SinkOperatorExecutor} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddInputChannelHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddInputChannelHandler.scala index e4ef003198b..585590208bd 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddInputChannelHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddInputChannelHandler.scala @@ -1,13 +1,13 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AddInputChannelRequest, AsyncRPCContext } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.{PAUSED, READY, RUNNING} +import edu.uci.ics.amber.engine.architecture.worker.WorkerState.{PAUSED, READY, RUNNING} trait AddInputChannelHandler { this: DataProcessorRPCHandlerInitializer => diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddPartitioningHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddPartitioningHandler.scala index c4cff88533f..31efb38f380 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddPartitioningHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddPartitioningHandler.scala @@ -1,13 +1,13 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AddPartitioningRequest, AsyncRPCContext } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.{PAUSED, READY, RUNNING} +import edu.uci.ics.amber.engine.architecture.worker.WorkerState.{PAUSED, READY, RUNNING} trait AddPartitioningHandler { this: DataProcessorRPCHandlerInitializer => diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AssignPortHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AssignPortHandler.scala index a3a98c8348d..08a721ba66a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AssignPortHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AssignPortHandler.scala @@ -1,11 +1,11 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AssignPortRequest, AsyncRPCContext } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer import edu.uci.ics.amber.engine.common.model.tuple.Schema diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/FinalizeCheckpointHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/FinalizeCheckpointHandler.scala index 6eb3a19e7a2..5a86a66614e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/FinalizeCheckpointHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/FinalizeCheckpointHandler.scala @@ -1,11 +1,11 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, FinalizeCheckpointRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse +import edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse import edu.uci.ics.amber.engine.architecture.worker.{ DataProcessorRPCHandlerInitializer, WorkflowWorker diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/FlushNetworkBufferHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/FlushNetworkBufferHandler.scala index 6237cac7293..aae4d79438b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/FlushNetworkBufferHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/FlushNetworkBufferHandler.scala @@ -1,9 +1,9 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext +import edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer trait FlushNetworkBufferHandler { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/InitializeExecutorHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/InitializeExecutorHandler.scala index d271f830a0e..ae87dcad52a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/InitializeExecutorHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/InitializeExecutorHandler.scala @@ -3,11 +3,11 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo.generateJavaOpExec -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, InitializeExecutorRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer import edu.uci.ics.amber.engine.common.{AmberRuntime, VirtualIdentityUtils} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/OpenExecutorHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/OpenExecutorHandler.scala index 7b596cab950..fe9a6c27b84 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/OpenExecutorHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/OpenExecutorHandler.scala @@ -1,8 +1,8 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{AsyncRPCContext, EmptyRequest} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, EmptyRequest} +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer trait OpenExecutorHandler { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/PauseHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/PauseHandler.scala index d1ac7e54247..10ed6a82818 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/PauseHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/PauseHandler.scala @@ -1,10 +1,10 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{AsyncRPCContext, EmptyRequest} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, EmptyRequest} +import edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse import edu.uci.ics.amber.engine.architecture.worker.{DataProcessorRPCHandlerInitializer, UserPause} -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.{PAUSED, READY, RUNNING} +import edu.uci.ics.amber.engine.architecture.worker.WorkerState.{PAUSED, READY, RUNNING} trait PauseHandler { this: DataProcessorRPCHandlerInitializer => diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/PrepareCheckpointHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/PrepareCheckpointHandler.scala index 26efb32e964..e05d650d780 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/PrepareCheckpointHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/PrepareCheckpointHandler.scala @@ -1,11 +1,11 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, PrepareCheckpointRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.{ DataProcessorRPCHandlerInitializer, WorkflowWorker @@ -13,7 +13,7 @@ import edu.uci.ics.amber.engine.architecture.worker.{ import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.MainThreadDelegateMessage import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage import edu.uci.ics.amber.engine.common.{CheckpointState, CheckpointSupport, SerializedState} -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity +import edu.uci.ics.amber.engine.common.ChannelMarkerIdentity import java.util.concurrent.CompletableFuture import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/QueryStatisticsHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/QueryStatisticsHandler.scala index 19d1b308841..43a53f6e060 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/QueryStatisticsHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/QueryStatisticsHandler.scala @@ -1,9 +1,9 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{EmptyRequest, AsyncRPCContext} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics +import edu.uci.ics.amber.engine.architecture.rpc.{EmptyRequest, AsyncRPCContext} +import edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse +import edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer trait QueryStatisticsHandler { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/ResumeHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/ResumeHandler.scala index 4bfe8f1ec54..d27a6828583 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/ResumeHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/ResumeHandler.scala @@ -1,10 +1,10 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{EmptyRequest, AsyncRPCContext} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse +import edu.uci.ics.amber.engine.architecture.rpc.{EmptyRequest, AsyncRPCContext} +import edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse import edu.uci.ics.amber.engine.architecture.worker.{DataProcessorRPCHandlerInitializer, UserPause} -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.{PAUSED, RUNNING} +import edu.uci.ics.amber.engine.architecture.worker.WorkerState.{PAUSED, RUNNING} trait ResumeHandler { this: DataProcessorRPCHandlerInitializer => diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/RetrieveStateHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/RetrieveStateHandler.scala index 57acd477e3f..11f359055e7 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/RetrieveStateHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/RetrieveStateHandler.scala @@ -1,8 +1,8 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{EmptyRequest, AsyncRPCContext} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.rpc.{EmptyRequest, AsyncRPCContext} +import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer trait RetrieveStateHandler { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/StartHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/StartHandler.scala index 92f8eb1e1a1..e2ce9f4304a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/StartHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/StartHandler.scala @@ -1,17 +1,17 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{EmptyRequest, AsyncRPCContext} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse +import edu.uci.ics.amber.engine.architecture.rpc.{EmptyRequest, AsyncRPCContext} +import edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.{READY, RUNNING} +import edu.uci.ics.amber.engine.architecture.worker.WorkerState.{READY, RUNNING} import edu.uci.ics.amber.engine.common.amberexception.WorkflowRuntimeException import edu.uci.ics.amber.engine.common.ambermessage.MarkerFrame import edu.uci.ics.amber.engine.common.executor.SourceOperatorExecutor import edu.uci.ics.amber.engine.common.model.{EndOfInputChannel, StartOfInputChannel} -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity -import edu.uci.ics.amber.engine.common.virtualidentity.util.SOURCE_STARTER_ACTOR -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.ChannelIdentity +import edu.uci.ics.amber.engine.common.util.SOURCE_STARTER_ACTOR +import edu.uci.ics.amber.engine.common.PortIdentity trait StartHandler { this: DataProcessorRPCHandlerInitializer => diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/AmberLogging.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/AmberLogging.scala index 982ec5f831f..eb81ac4683c 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/AmberLogging.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/AmberLogging.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.common import com.typesafe.scalalogging.Logger -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import org.slf4j.LoggerFactory trait AmberLogging { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/CheckpointSupport.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/CheckpointSupport.scala index 04552bec37e..83d3748b315 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/CheckpointSupport.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/CheckpointSupport.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.common import edu.uci.ics.amber.engine.common.model.tuple.TupleLike -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity trait CheckpointSupport { def serializeState( diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/Utils.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/Utils.scala index 933157da3bf..ca07208a4fe 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/Utils.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/Utils.scala @@ -5,7 +5,6 @@ import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.noctordeser.NoCtorDeserModule import com.fasterxml.jackson.module.scala.DefaultScalaModule import com.typesafe.scalalogging.LazyLogging -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState import org.jooq.DSLContext import org.jooq.impl.DSL diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/VirtualIdentityUtils.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/VirtualIdentityUtils.scala index 937cc100737..23497321574 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/VirtualIdentityUtils.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/VirtualIdentityUtils.scala @@ -1,12 +1,5 @@ package edu.uci.ics.amber.engine.common -import edu.uci.ics.amber.engine.common.virtualidentity.{ - ActorVirtualIdentity, - OperatorIdentity, - PhysicalOpIdentity, - WorkflowIdentity -} - import scala.util.matching.Regex object VirtualIdentityUtils { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/amberexception/WorkflowRuntimeException.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/amberexception/WorkflowRuntimeException.scala index d75a9e5eb16..ea20e6737a2 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/amberexception/WorkflowRuntimeException.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/amberexception/WorkflowRuntimeException.scala @@ -1,6 +1,6 @@ package edu.uci.ics.amber.engine.common.amberexception -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity class WorkflowRuntimeException( val message: String, diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/ambermessage/RecoveryPayload.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/ambermessage/RecoveryPayload.scala index be6138ebd05..bcf4d07c6fd 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/ambermessage/RecoveryPayload.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/ambermessage/RecoveryPayload.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.common.ambermessage import akka.actor.{ActorRef, Address} -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity sealed trait RecoveryPayload extends Serializable {} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/ambermessage/WorkflowMessage.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/ambermessage/WorkflowMessage.scala index ad308f7a9fb..8df578a53cb 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/ambermessage/WorkflowMessage.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/ambermessage/WorkflowMessage.scala @@ -1,6 +1,6 @@ package edu.uci.ics.amber.engine.common.ambermessage -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} case object WorkflowMessage { def getInMemSize(msg: WorkflowMessage): Long = { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/client/AmberClient.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/client/AmberClient.scala index 77c6cee3ad0..252d718c835 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/client/AmberClient.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/client/AmberClient.scala @@ -5,9 +5,9 @@ import akka.pattern._ import akka.util.Timeout import com.twitter.util.{Future, Promise} import edu.uci.ics.amber.engine.architecture.controller.ControllerConfig -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest -import edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceFs2Grpc -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn +import edu.uci.ics.amber.engine.architecture.rpc.ControlRequest +import edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceFs2Grpc +import edu.uci.ics.amber.engine.architecture.rpc.ControlReturn import edu.uci.ics.amber.engine.common.FutureBijection._ import edu.uci.ics.amber.engine.common.ambermessage.{NotifyFailedNode, WorkflowRecoveryMessage} import edu.uci.ics.amber.engine.common.client.ClientActor.{ @@ -16,7 +16,7 @@ import edu.uci.ics.amber.engine.common.client.ClientActor.{ ObservableRequest } import edu.uci.ics.amber.engine.common.model.{PhysicalPlan, WorkflowContext} -import edu.uci.ics.amber.engine.common.virtualidentity.util.CLIENT +import edu.uci.ics.amber.engine.common.util.CLIENT import edu.uci.ics.texera.workflow.common.storage.OpResultStorage import io.reactivex.rxjava3.core.Observable import io.reactivex.rxjava3.disposables.Disposable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/client/ClientActor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/client/ClientActor.scala index 6cc30914c57..6ff276869a6 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/client/ClientActor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/client/ClientActor.scala @@ -10,9 +10,9 @@ import edu.uci.ics.amber.engine.architecture.common.WorkflowActor.{ NetworkMessage } import edu.uci.ics.amber.engine.architecture.controller.{ClientEvent, Controller, ControllerConfig} -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{AsyncRPCContext, ControlRequest} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.{ - ControlError, +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, ControlRequest} +import edu.uci.ics.amber.engine.architecture.rpc.{ + ControlException, ControlReturn, ReturnInvocation } @@ -32,8 +32,8 @@ import edu.uci.ics.amber.engine.common.client.ClientActor.{ } import edu.uci.ics.amber.engine.common.model.{PhysicalPlan, WorkflowContext} import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient -import edu.uci.ics.amber.engine.common.virtualidentity.util.{CLIENT, CONTROLLER} -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.util.{CLIENT, CONTROLLER} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} import edu.uci.ics.amber.error.ErrorUtils.reconstructThrowable import edu.uci.ics.texera.workflow.common.storage.OpResultStorage @@ -112,7 +112,7 @@ private[client] class ClientActor extends Actor with AmberLogging { case ReturnInvocation(originalCommandID, controlReturn) => if (promiseMap.contains(originalCommandID)) { controlReturn match { - case t: ControlError => + case t: ControlException => promiseMap(originalCommandID).setException(reconstructThrowable(t)) case other => promiseMap(originalCommandID).setValue(other) diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/executor/OperatorExecutor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/executor/OperatorExecutor.scala index dfa512a0dfe..02e98607a2d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/executor/OperatorExecutor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/executor/OperatorExecutor.scala @@ -2,7 +2,7 @@ package edu.uci.ics.amber.engine.common.executor import edu.uci.ics.amber.engine.common.model.State import edu.uci.ics.amber.engine.common.model.tuple.{Tuple, TupleLike} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity trait OperatorExecutor { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/executor/SinkOperatorExecutor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/executor/SinkOperatorExecutor.scala index 4179b933e8b..c2381de3df9 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/executor/SinkOperatorExecutor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/executor/SinkOperatorExecutor.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.common.executor import edu.uci.ics.amber.engine.common.model.tuple.{Tuple, TupleLike} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity trait SinkOperatorExecutor extends OperatorExecutor { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/executor/SourceOperatorExecutor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/executor/SourceOperatorExecutor.scala index 453a41c5ac6..f10a9a1b5ca 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/executor/SourceOperatorExecutor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/executor/SourceOperatorExecutor.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.common.executor import edu.uci.ics.amber.engine.common.model.tuple.{Tuple, TupleLike} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity trait SourceOperatorExecutor extends OperatorExecutor { override def open(): Unit = {} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/PhysicalOp.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/PhysicalOp.scala index f8379d8bd08..b10e44649f2 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/PhysicalOp.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/PhysicalOp.scala @@ -26,8 +26,7 @@ import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.{ } import edu.uci.ics.amber.engine.common.VirtualIdentityUtils import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity._ -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PhysicalLink, PortIdentity} +import edu.uci.ics.amber.engine.common._ import edu.uci.ics.texera.workflow.common.workflow._ import org.jgrapht.graph.{DefaultEdge, DirectedAcyclicGraph} import org.jgrapht.traverse.TopologicalOrderIterator diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/PhysicalPlan.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/PhysicalPlan.scala index 5e7f1577a60..b79c8358691 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/PhysicalPlan.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/PhysicalPlan.scala @@ -3,12 +3,12 @@ package edu.uci.ics.amber.engine.common.model import com.fasterxml.jackson.annotation.JsonIgnore import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.common.VirtualIdentityUtils -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.common.{ ActorVirtualIdentity, OperatorIdentity, PhysicalOpIdentity } -import edu.uci.ics.amber.engine.common.workflow.PhysicalLink +import edu.uci.ics.amber.engine.common.PhysicalLink import edu.uci.ics.texera.workflow.common.workflow.{LogicalPlan, PartitionInfo, UnknownPartition} import org.jgrapht.alg.connectivity.BiconnectivityInspector import org.jgrapht.alg.shortestpath.AllDirectedPaths diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/WorkflowContext.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/WorkflowContext.scala index d1443257fa0..4d014691352 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/WorkflowContext.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/WorkflowContext.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.WorkflowContext.{ DEFAULT_WORKFLOW_ID, DEFAULT_WORKFLOW_SETTINGS } -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} object WorkflowContext { val DEFAULT_EXECUTION_ID: ExecutionIdentity = ExecutionIdentity(1L) diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/tuple/TupleLike.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/tuple/TupleLike.scala index 7c00264cf6e..e8de5410682 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/tuple/TupleLike.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/tuple/TupleLike.scala @@ -1,6 +1,6 @@ package edu.uci.ics.amber.engine.common.model.tuple -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity import scala.jdk.CollectionConverters.CollectionHasAsScala diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/rpc/AsyncRPCClient.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/rpc/AsyncRPCClient.scala index ffa8962acd4..df444a63416 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/rpc/AsyncRPCClient.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/rpc/AsyncRPCClient.scala @@ -4,28 +4,28 @@ import com.twitter.util.{Future, Promise} import edu.uci.ics.amber.engine.architecture.controller.ClientEvent import edu.uci.ics.amber.engine.architecture.messaginglayer.NetworkOutputGateway import edu.uci.ics.amber.engine.common.AmberLogging -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.common.{ ActorVirtualIdentity, ChannelIdentity, ChannelMarkerIdentity } -import edu.uci.ics.amber.engine.common.virtualidentity.util.CLIENT +import edu.uci.ics.amber.engine.common.util.CLIENT import io.grpc.MethodDescriptor -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, ChannelMarkerPayload, ChannelMarkerType, ControlInvocation, ControlRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceFs2Grpc -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.{ - ControlError, +import edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceFs2Grpc +import edu.uci.ics.amber.engine.architecture.rpc.{ + ControlException, ControlReturn, ReturnInvocation, WorkerMetricsResponse } -import edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceFs2Grpc +import edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceFs2Grpc import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient.createProxy import edu.uci.ics.amber.error.ErrorUtils.reconstructThrowable @@ -166,7 +166,7 @@ class AsyncRPCClient( if (unfulfilledPromises.contains(ret.commandId)) { val p = unfulfilledPromises(ret.commandId) ret.returnValue match { - case err: ControlError => + case err: ControlException => p.raise(reconstructThrowable(err)) case other => p.setValue(other) @@ -187,7 +187,7 @@ class AsyncRPCClient( s"receive reply: ${ret.returnValue.getClass.getSimpleName} from $channelId (controlID: ${ret.commandId})" ) ret.returnValue match { - case err: ControlError => + case err: ControlException => logger.error(s"received error from $channelId", err) case _ => } diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/rpc/AsyncRPCHandlerInitializer.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/rpc/AsyncRPCHandlerInitializer.scala index 149c1e423ad..b380ee44d24 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/rpc/AsyncRPCHandlerInitializer.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/rpc/AsyncRPCHandlerInitializer.scala @@ -2,11 +2,8 @@ package edu.uci.ics.amber.engine.common.rpc import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ClientEvent -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands._ -import edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceFs2Grpc -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns._ -import edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceFs2Grpc -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.architecture.rpc._ +import edu.uci.ics.amber.engine.common.{ ActorVirtualIdentity, ChannelIdentity, ChannelMarkerIdentity diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/rpc/AsyncRPCServer.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/rpc/AsyncRPCServer.scala index 326537b72f9..c33162787b9 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/rpc/AsyncRPCServer.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/rpc/AsyncRPCServer.scala @@ -2,14 +2,14 @@ package edu.uci.ics.amber.engine.common.rpc import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.messaginglayer.NetworkOutputGateway -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AsyncRPCContext, ControlInvocation, ControlRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.{ControlReturn, ReturnInvocation} +import edu.uci.ics.amber.engine.architecture.rpc.{ControlReturn, ReturnInvocation} import edu.uci.ics.amber.engine.common.AmberLogging -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import edu.uci.ics.amber.error.ErrorUtils.mkControlError import java.lang.reflect.Method diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/statetransition/StateManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/statetransition/StateManager.scala index 39291b5c9d3..6c3390aadf8 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/statetransition/StateManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/statetransition/StateManager.scala @@ -5,7 +5,7 @@ import edu.uci.ics.amber.engine.common.statetransition.StateManager.{ InvalidStateException, InvalidTransitionException } -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity object StateManager { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/statetransition/WorkerStateManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/statetransition/WorkerStateManager.scala index a931e77f087..ace20681b35 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/statetransition/WorkerStateManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/statetransition/WorkerStateManager.scala @@ -1,8 +1,8 @@ package edu.uci.ics.amber.engine.common.statetransition -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState._ -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.architecture.worker.WorkerState +import edu.uci.ics.amber.engine.architecture.worker.WorkerState._ +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity // The following pattern is a good practice of enum in scala // We've always used this pattern in the codebase diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/storage/SequentialRecordStorage.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/storage/SequentialRecordStorage.scala index d681348044c..8ad652573b0 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/storage/SequentialRecordStorage.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/storage/SequentialRecordStorage.scala @@ -8,7 +8,7 @@ import edu.uci.ics.amber.engine.architecture.logreplay.{ ReplayLogRecord } import SequentialRecordStorage.{SequentialRecordReader, SequentialRecordWriter} -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState +import edu.uci.ics.amber.engine.architecture.worker.WorkerState import java.io.{DataInputStream, DataOutputStream} import java.net.URI diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/virtualidentity/util.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/util.scala similarity index 89% rename from core/amber/src/main/scala/edu/uci/ics/amber/engine/common/virtualidentity/util.scala rename to core/amber/src/main/scala/edu/uci/ics/amber/engine/common/util.scala index fd5ac18d658..04cfcea30a3 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/virtualidentity/util.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/util.scala @@ -1,4 +1,4 @@ -package edu.uci.ics.amber.engine.common.virtualidentity +package edu.uci.ics.amber.engine.common object util { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/error/ErrorUtils.scala b/core/amber/src/main/scala/edu/uci/ics/amber/error/ErrorUtils.scala index 1bdc2909e4a..32a3a290cd9 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/error/ErrorUtils.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/error/ErrorUtils.scala @@ -1,11 +1,11 @@ package edu.uci.ics.amber.error import com.google.protobuf.timestamp.Timestamp -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType.ERROR -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.{ControlError, ErrorLanguage} +import edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage +import edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType.ERROR +import edu.uci.ics.amber.engine.architecture.rpc.{ControlException, ErrorLanguage} import edu.uci.ics.amber.engine.common.VirtualIdentityUtils -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import java.time.Instant import scala.util.control.ControlThrowable @@ -37,12 +37,12 @@ object ErrorUtils { ConsoleMessage(actorId.name, Timestamp(Instant.now), ERROR, source, title, message) } - def mkControlError(err: Throwable): ControlError = { + def mkControlError(err: Throwable): ControlException = { val stacktrace = err.getStackTrace.mkString("\n") - ControlError(err.toString, err.getCause.toString, stacktrace, ErrorLanguage.SCALA) + ControlException(err.toString, err.getCause.toString, stacktrace, ErrorLanguage.SCALA) } - def reconstructThrowable(controlError: ControlError): Throwable = { + def reconstructThrowable(controlError: ControlException): Throwable = { if (controlError.language == ErrorLanguage.PYTHON) { return new Throwable(controlError.errorMessage) } else { diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/TexeraWebApplication.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/TexeraWebApplication.scala index 90dfffbec6a..5c1e97919db 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/TexeraWebApplication.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/TexeraWebApplication.scala @@ -6,24 +6,16 @@ import com.github.dirkraft.dropwizard.fileassets.FileAssetsBundle import com.github.toastshaman.dropwizard.auth.jwt.JwtAuthFilter import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.architecture.controller.ControllerConfig -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.{ - COMPLETED, - FAILED -} import edu.uci.ics.amber.engine.common.AmberRuntime.scheduleRecurringCallThroughActorSystem import edu.uci.ics.amber.engine.common.{AmberConfig, AmberRuntime, Utils} import edu.uci.ics.amber.engine.common.client.AmberClient import edu.uci.ics.amber.engine.common.model.{PhysicalPlan, WorkflowContext} import edu.uci.ics.amber.engine.common.storage.SequentialRecordStorage -import edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity +import edu.uci.ics.amber.engine.common.ExecutionIdentity import Utils.{maptoStatusCode, objectMapper} +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState.{COMPLETED, FAILED} import edu.uci.ics.texera.web.auth.JwtAuth.jwtConsumer -import edu.uci.ics.texera.web.auth.{ - GuestAuthFilter, - SessionUser, - UserAuthenticator, - UserRoleAuthorizer -} +import edu.uci.ics.texera.web.auth.{GuestAuthFilter, SessionUser, UserAuthenticator, UserRoleAuthorizer} import edu.uci.ics.texera.web.model.jooq.generated.tables.pojos.WorkflowExecutions import edu.uci.ics.texera.web.resource.auth.{AuthResource, GoogleAuthResource} import edu.uci.ics.texera.web.resource._ @@ -31,29 +23,14 @@ import edu.uci.ics.texera.web.resource.dashboard.DashboardResource import edu.uci.ics.texera.web.resource.dashboard.admin.execution.AdminExecutionResource import edu.uci.ics.texera.web.resource.dashboard.admin.user.AdminUserResource import edu.uci.ics.texera.web.resource.dashboard.hub.workflow.HubWorkflowResource -import edu.uci.ics.texera.web.resource.dashboard.user.dataset.{ - DatasetAccessResource, - DatasetResource -} -import edu.uci.ics.texera.web.resource.dashboard.user.dataset.`type`.{ - DatasetFileNode, - DatasetFileNodeSerializer -} +import edu.uci.ics.texera.web.resource.dashboard.user.dataset.{DatasetAccessResource, DatasetResource} +import edu.uci.ics.texera.web.resource.dashboard.user.dataset.`type`.{DatasetFileNode, DatasetFileNodeSerializer} import edu.uci.ics.texera.web.resource.dashboard.user.dataset.service.GitVersionControlLocalFileStorage import edu.uci.ics.texera.web.resource.dashboard.user.dataset.utils.PathUtils.getAllDatasetDirectories -import edu.uci.ics.texera.web.resource.dashboard.user.project.{ - ProjectAccessResource, - ProjectResource, - PublicProjectResource -} +import edu.uci.ics.texera.web.resource.dashboard.user.project.{ProjectAccessResource, ProjectResource, PublicProjectResource} import edu.uci.ics.texera.web.resource.dashboard.user.quota.UserQuotaResource import edu.uci.ics.texera.web.resource.dashboard.user.discussion.UserDiscussionResource -import edu.uci.ics.texera.web.resource.dashboard.user.workflow.{ - WorkflowAccessResource, - WorkflowExecutionsResource, - WorkflowResource, - WorkflowVersionResource -} +import edu.uci.ics.texera.web.resource.dashboard.user.workflow.{WorkflowAccessResource, WorkflowExecutionsResource, WorkflowResource, WorkflowVersionResource} import edu.uci.ics.texera.web.resource.languageserver.PythonLanguageServerManager import edu.uci.ics.texera.web.service.ExecutionsMetadataPersistService import edu.uci.ics.texera.web.storage.MongoDatabaseManager diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/WorkflowLifecycleManager.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/WorkflowLifecycleManager.scala index 5d38687b242..66ff9ac316a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/WorkflowLifecycleManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/WorkflowLifecycleManager.scala @@ -3,11 +3,9 @@ package edu.uci.ics.texera.web import java.time.{LocalDateTime, Duration => JDuration} import akka.actor.Cancellable import com.typesafe.scalalogging.LazyLogging -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.RUNNING -import edu.uci.ics.amber.engine.common.AmberRuntime +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState.RUNNING +import edu.uci.ics.amber.engine.common.{AmberRuntime, ExecutionMetadataStore, WorkflowAggregatedState} import edu.uci.ics.texera.web.storage.ExecutionStateStore -import edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore import scala.concurrent.duration.DurationInt diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/event/WorkflowErrorEvent.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/event/WorkflowErrorEvent.scala index 5afcb94f637..50309f8cf81 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/event/WorkflowErrorEvent.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/event/WorkflowErrorEvent.scala @@ -1,6 +1,6 @@ package edu.uci.ics.texera.web.model.websocket.event -import edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError +import edu.uci.ics.amber.engine.common.WorkflowFatalError case class WorkflowErrorEvent( fatalErrors: Seq[WorkflowFatalError] diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/event/python/ConsoleUpdateEvent.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/event/python/ConsoleUpdateEvent.scala index cdcf2be71db..2310287c6a1 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/event/python/ConsoleUpdateEvent.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/event/python/ConsoleUpdateEvent.scala @@ -1,6 +1,6 @@ package edu.uci.ics.texera.web.model.websocket.event.python -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage +import edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage import edu.uci.ics.texera.web.model.websocket.event.TexeraWebSocketEvent object ConsoleUpdateEvent {} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/response/python/PythonExpressionEvaluateResponse.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/response/python/PythonExpressionEvaluateResponse.scala index 10712538a33..39ffad45b2e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/response/python/PythonExpressionEvaluateResponse.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/response/python/PythonExpressionEvaluateResponse.scala @@ -1,6 +1,6 @@ package edu.uci.ics.texera.web.model.websocket.response.python -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue +import edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue import edu.uci.ics.texera.web.model.websocket.event.TexeraWebSocketEvent case class PythonExpressionEvaluateResponse( diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/SchemaPropagationResource.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/SchemaPropagationResource.scala index dfcf38cedf3..8031920b844 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/SchemaPropagationResource.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/SchemaPropagationResource.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.web.resource import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.common.Utils import edu.uci.ics.amber.engine.common.model.{PhysicalPlan, WorkflowContext} -import edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity +import edu.uci.ics.amber.engine.common.WorkflowIdentity import edu.uci.ics.texera.web.auth.SessionUser import edu.uci.ics.texera.web.model.http.response.SchemaPropagationResponse import edu.uci.ics.texera.web.model.websocket.request.LogicalPlanPojo diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowCompilationResource.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowCompilationResource.scala index 7c314ec5fe5..2490f5085eb 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowCompilationResource.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowCompilationResource.scala @@ -3,7 +3,7 @@ package edu.uci.ics.texera.web.resource import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.common.model.{PhysicalPlan, WorkflowContext} import edu.uci.ics.amber.engine.common.model.tuple.Attribute -import edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity +import edu.uci.ics.amber.engine.common.WorkflowIdentity import edu.uci.ics.texera.web.model.websocket.request.LogicalPlanPojo import edu.uci.ics.texera.workflow.common.workflow.WorkflowCompiler import org.jooq.types.UInteger diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowWebsocketResource.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowWebsocketResource.scala index 752b7a33fe3..9f62763d22a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowWebsocketResource.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowWebsocketResource.scala @@ -3,26 +3,19 @@ package edu.uci.ics.texera.web.resource import com.google.protobuf.timestamp.Timestamp import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.clustering.ClusterListener -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.{ - PAUSED, - RUNNING -} import edu.uci.ics.amber.engine.common.model.WorkflowContext -import edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity +import edu.uci.ics.amber.engine.common.WorkflowIdentity import edu.uci.ics.amber.error.ErrorUtils.getStackTraceWithAllCauses import edu.uci.ics.amber.engine.common.Utils.objectMapper import edu.uci.ics.texera.web.model.jooq.generated.tables.pojos.User -import edu.uci.ics.texera.web.model.websocket.event.{ - CacheStatusUpdateEvent, - WorkflowErrorEvent, - WorkflowStateEvent -} +import edu.uci.ics.texera.web.model.websocket.event.{CacheStatusUpdateEvent, WorkflowErrorEvent, WorkflowStateEvent} import edu.uci.ics.texera.web.model.websocket.request._ import edu.uci.ics.texera.web.model.websocket.response._ import edu.uci.ics.texera.web.service.{WorkflowCacheChecker, WorkflowService} import edu.uci.ics.texera.web.storage.ExecutionStateStore -import edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.COMPILATION_ERROR -import edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError +import edu.uci.ics.amber.engine.common.FatalErrorType.COMPILATION_ERROR +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState._ +import edu.uci.ics.amber.engine.common.WorkflowFatalError import edu.uci.ics.texera.web.{ServletAwareConfigurator, SessionState} import edu.uci.ics.texera.workflow.common.workflow.WorkflowCompiler diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/dashboard/user/workflow/WorkflowExecutionsResource.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/dashboard/user/workflow/WorkflowExecutionsResource.scala index d458a8a8e24..db0e2e615ac 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/dashboard/user/workflow/WorkflowExecutionsResource.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/dashboard/user/workflow/WorkflowExecutionsResource.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.web.resource.dashboard.user.workflow import edu.uci.ics.amber.engine.architecture.logreplay.{ReplayDestination, ReplayLogRecord} import edu.uci.ics.amber.engine.common.storage.SequentialRecordStorage -import edu.uci.ics.amber.engine.common.virtualidentity.{ChannelMarkerIdentity, ExecutionIdentity} +import edu.uci.ics.amber.engine.common.{ChannelMarkerIdentity, ExecutionIdentity} import edu.uci.ics.texera.web.SqlServer import edu.uci.ics.texera.web.auth.SessionUser import edu.uci.ics.texera.web.model.jooq.generated.Tables.{ diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/EmailNotificationService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/EmailNotificationService.scala index 1f6f500f636..954284179bf 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/EmailNotificationService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/EmailNotificationService.scala @@ -1,6 +1,7 @@ package edu.uci.ics.texera.web.service -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState + +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState import java.util.concurrent.{ExecutorService, Executors} import scala.concurrent.{ExecutionContext, Future} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionConsoleService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionConsoleService.scala index e1e62df5f98..e7c9747209b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionConsoleService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionConsoleService.scala @@ -2,15 +2,15 @@ package edu.uci.ics.texera.web.service import com.google.protobuf.timestamp.Timestamp import com.twitter.util.{Await, Duration} -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ DebugCommandRequest => AmberDebugCommandRequest, ConsoleMessage, EvaluatePythonExpressionRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType.COMMAND +import edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType.COMMAND import edu.uci.ics.amber.engine.common.{AmberConfig, VirtualIdentityUtils} import edu.uci.ics.amber.engine.common.client.AmberClient -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import edu.uci.ics.texera.web.model.websocket.event.TexeraWebSocketEvent import edu.uci.ics.texera.web.model.websocket.event.python.ConsoleUpdateEvent import edu.uci.ics.texera.web.model.websocket.request.python.{ @@ -19,7 +19,7 @@ import edu.uci.ics.texera.web.model.websocket.request.python.{ } import edu.uci.ics.texera.web.model.websocket.response.python.PythonExpressionEvaluateResponse import edu.uci.ics.texera.web.storage.ExecutionStateStore -import edu.uci.ics.amber.engine.common.workflowruntimestate.{ +import edu.uci.ics.amber.engine.common.{ EvaluatedValueList, ExecutionConsoleStore, OperatorConsole diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionResultService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionResultService.scala index ef3ed1763c8..af1ae2bafd7 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionResultService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionResultService.scala @@ -5,31 +5,18 @@ import com.fasterxml.jackson.annotation.{JsonTypeInfo, JsonTypeName} import com.fasterxml.jackson.databind.node.ObjectNode import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.architecture.controller.{ExecutionStateUpdate, FatalError} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.{ - COMPLETED, - FAILED, - KILLED -} -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.RUNNING +import edu.uci.ics.amber.engine.architecture.worker.WorkerState.RUNNING import edu.uci.ics.amber.engine.common.{AmberConfig, AmberRuntime, IncrementalOutputMode} import edu.uci.ics.amber.engine.common.client.AmberClient import edu.uci.ics.amber.engine.common.model.tuple.Tuple -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity +import edu.uci.ics.amber.engine.common.OperatorIdentity import edu.uci.ics.amber.engine.common.IncrementalOutputMode.{SET_DELTA, SET_SNAPSHOT} -import edu.uci.ics.texera.web.model.websocket.event.{ - PaginatedResultEvent, - TexeraWebSocketEvent, - WebResultUpdateEvent -} +import edu.uci.ics.texera.web.model.websocket.event.{PaginatedResultEvent, TexeraWebSocketEvent, WebResultUpdateEvent} import edu.uci.ics.texera.web.model.websocket.request.ResultPaginationRequest import edu.uci.ics.texera.web.service.ExecutionResultService.WebResultUpdate -import edu.uci.ics.texera.web.storage.{ - ExecutionStateStore, - OperatorResultMetadata, - WorkflowResultStore, - WorkflowStateStore -} -import edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore +import edu.uci.ics.texera.web.storage.{ExecutionStateStore, OperatorResultMetadata, WorkflowResultStore, WorkflowStateStore} +import edu.uci.ics.amber.engine.common.ExecutionMetadataStore +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState._ import edu.uci.ics.texera.web.SubscriptionManager import edu.uci.ics.texera.workflow.common.storage.OpResultStorage import edu.uci.ics.texera.workflow.common.workflow.LogicalPlan diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionRuntimeService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionRuntimeService.scala index 3e64292c08d..8018e9adddc 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionRuntimeService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionRuntimeService.scala @@ -2,22 +2,13 @@ package edu.uci.ics.texera.web.service import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.architecture.controller.ExecutionStateUpdate -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ - EmptyRequest, - TakeGlobalCheckpointRequest -} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState._ +import edu.uci.ics.amber.engine.architecture.rpc.{EmptyRequest, TakeGlobalCheckpointRequest} import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.FaultToleranceConfig import edu.uci.ics.amber.engine.common.client.AmberClient -import edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity +import edu.uci.ics.amber.engine.common.ChannelMarkerIdentity +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState._ import edu.uci.ics.texera.web.{SubscriptionManager, WebsocketInput} -import edu.uci.ics.texera.web.model.websocket.request.{ - SkipTupleRequest, - WorkflowCheckpointRequest, - WorkflowKillRequest, - WorkflowPauseRequest, - WorkflowResumeRequest -} +import edu.uci.ics.texera.web.model.websocket.request.{SkipTupleRequest, WorkflowCheckpointRequest, WorkflowKillRequest, WorkflowPauseRequest, WorkflowResumeRequest} import edu.uci.ics.texera.web.storage.ExecutionStateStore import edu.uci.ics.texera.web.storage.ExecutionStateStore.updateWorkflowState @@ -44,7 +35,7 @@ class ExecutionRuntimeService( ) if (evt.state == COMPLETED) { client.shutdown() - stateStore.statsStore.updateState(stats => stats.withEndTimeStamp(System.currentTimeMillis())) + stateStore.statsStore.updateState(stats => stats.withEndTimestamp(System.currentTimeMillis())) } })) @@ -74,7 +65,7 @@ class ExecutionRuntimeService( // Receive Kill addSubscription(wsInput.subscribe((req: WorkflowKillRequest, uidOpt) => { client.shutdown() - stateStore.statsStore.updateState(stats => stats.withEndTimeStamp(System.currentTimeMillis())) + stateStore.statsStore.updateState(stats => stats.withEndTimestamp(System.currentTimeMillis())) stateStore.metadataStore.updateState(metadataStore => updateWorkflowState(KILLED, metadataStore) ) diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala index 81ee2678e04..ca14763469f 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala @@ -2,16 +2,9 @@ package edu.uci.ics.texera.web.service import com.google.protobuf.timestamp.Timestamp import com.typesafe.scalalogging.LazyLogging -import edu.uci.ics.amber.engine.architecture.controller.{ - ExecutionStatsUpdate, - FatalError, - WorkerAssignmentUpdate, - WorkflowRecoveryStatus -} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.FAILED -import edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping -import edu.uci.ics.amber.engine.common.{AmberConfig, Utils} +import edu.uci.ics.amber.engine.architecture.controller.{ExecutionStatsUpdate, FatalError, WorkerAssignmentUpdate, WorkflowRecoveryStatus} +import edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping +import edu.uci.ics.amber.engine.common.{AmberConfig, OperatorMetrics, OperatorStatistics, OperatorWorkerMapping, Utils, WorkflowAggregatedState, WorkflowFatalError} import edu.uci.ics.amber.engine.common.client.AmberClient import edu.uci.ics.amber.engine.common.model.WorkflowContext import edu.uci.ics.amber.error.ErrorUtils.{getOperatorFromActorIdOpt, getStackTraceWithAllCauses} @@ -19,21 +12,11 @@ import Utils.maptoStatusCode import edu.uci.ics.texera.web.model.jooq.generated.tables.pojos.WorkflowRuntimeStatistics import edu.uci.ics.texera.web.model.jooq.generated.tables.daos.WorkflowRuntimeStatisticsDao import edu.uci.ics.texera.web.{SqlServer, SubscriptionManager} -import edu.uci.ics.texera.web.model.websocket.event.{ - ExecutionDurationUpdateEvent, - OperatorAggregatedMetrics, - OperatorStatisticsUpdateEvent, - WorkerAssignmentUpdateEvent -} +import edu.uci.ics.texera.web.model.websocket.event.{ExecutionDurationUpdateEvent, OperatorAggregatedMetrics, OperatorStatisticsUpdateEvent, WorkerAssignmentUpdateEvent} import edu.uci.ics.texera.web.storage.ExecutionStateStore import edu.uci.ics.texera.web.storage.ExecutionStateStore.updateWorkflowState -import edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.EXECUTION_FAILURE -import edu.uci.ics.amber.engine.common.workflowruntimestate.{ - OperatorMetrics, - OperatorStatistics, - OperatorWorkerMapping, - WorkflowFatalError -} +import edu.uci.ics.amber.engine.common.FatalErrorType.EXECUTION_FAILURE +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState.FAILED import java.time.Instant import org.jooq.types.{UInteger, ULong} @@ -97,19 +80,19 @@ class ExecutionStatsService( stateStore.statsStore.registerDiffHandler((oldState, newState) => { // update execution duration. if ( - newState.startTimeStamp != oldState.startTimeStamp || newState.endTimeStamp != oldState.endTimeStamp + newState.startTimestamp != oldState.startTimestamp || newState.endTimestamp != oldState.endTimestamp ) { - if (newState.endTimeStamp != 0) { + if (newState.endTimestamp != 0) { Iterable( ExecutionDurationUpdateEvent( - newState.endTimeStamp - newState.startTimeStamp, + newState.endTimestamp - newState.startTimestamp, isRunning = false ) ) } else { val currentTime = System.currentTimeMillis() Iterable( - ExecutionDurationUpdateEvent(currentTime - newState.startTimeStamp, isRunning = true) + ExecutionDurationUpdateEvent(currentTime - newState.startTimestamp, isRunning = true) ) } } else { @@ -269,7 +252,7 @@ class ExecutionStatsService( client.shutdown() val (operatorId, workerId) = getOperatorFromActorIdOpt(evt.fromActor) stateStore.statsStore.updateState(stats => - stats.withEndTimeStamp(System.currentTimeMillis()) + stats.withEndTimestamp(System.currentTimeMillis()) ) stateStore.metadataStore.updateState { metadataStore => logger.error("error occurred in execution", evt.e) diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionsMetadataPersistService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionsMetadataPersistService.scala index 618f2aa619f..133e63f5bf5 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionsMetadataPersistService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionsMetadataPersistService.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.web.service import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.common.AmberConfig -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.web.SqlServer import edu.uci.ics.texera.web.model.jooq.generated.tables.daos.WorkflowExecutionsDao import edu.uci.ics.texera.web.model.jooq.generated.tables.pojos.WorkflowExecutions diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/FriesReconfigurationAlgorithm.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/FriesReconfigurationAlgorithm.scala index 451b75e4b0c..b60a4046d6c 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/FriesReconfigurationAlgorithm.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/FriesReconfigurationAlgorithm.scala @@ -1,12 +1,12 @@ package edu.uci.ics.texera.web.service -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ ModifyLogicRequest, PropagateChannelMarkerRequest } import edu.uci.ics.amber.engine.architecture.scheduling.{Region, WorkflowExecutionCoordinator} import edu.uci.ics.amber.engine.common.model.PhysicalPlan -import edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity +import edu.uci.ics.amber.engine.common.PhysicalOpIdentity import org.jgrapht.alg.connectivity.ConnectivityInspector import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ResultExportService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ResultExportService.scala index 86388122f1d..08f7aee6b20 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ResultExportService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ResultExportService.scala @@ -12,7 +12,7 @@ import com.google.api.services.drive.model.{File, FileList, Permission} import com.google.api.services.sheets.v4.Sheets import com.google.api.services.sheets.v4.model.{Spreadsheet, SpreadsheetProperties, ValueRange} import edu.uci.ics.amber.engine.common.model.tuple.Tuple -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity +import edu.uci.ics.amber.engine.common.OperatorIdentity import edu.uci.ics.amber.engine.common.Utils.retry import edu.uci.ics.texera.web.model.jooq.generated.tables.pojos.User import edu.uci.ics.texera.web.model.websocket.request.ResultExportRequest diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowCacheChecker.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowCacheChecker.scala index f9a9b1153cf..c83fe19034d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowCacheChecker.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowCacheChecker.scala @@ -1,6 +1,6 @@ package edu.uci.ics.texera.web.service -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity +import edu.uci.ics.amber.engine.common.OperatorIdentity import edu.uci.ics.texera.web.model.websocket.request.EditingTimeCompilationRequest import edu.uci.ics.texera.workflow.common.workflow.LogicalPlan diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowEmailNotifier.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowEmailNotifier.scala index aef3b04da5a..143f2d6661c 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowEmailNotifier.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowEmailNotifier.scala @@ -1,7 +1,7 @@ package edu.uci.ics.texera.web.service -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState._ +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState._ import edu.uci.ics.texera.web.resource.dashboard.user.workflow.WorkflowResource import edu.uci.ics.texera.web.resource.{EmailMessage, GmailResource} import org.hibernate.validator.internal.constraintvalidators.hv.EmailValidator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowExecutionService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowExecutionService.scala index b0b64f3d4f3..e898ca68e3e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowExecutionService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowExecutionService.scala @@ -2,20 +2,16 @@ package edu.uci.ics.texera.web.service import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.architecture.controller.{ControllerConfig, Workflow} -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState._ +import edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest import edu.uci.ics.amber.engine.common.Utils import edu.uci.ics.amber.engine.common.client.AmberClient import edu.uci.ics.amber.engine.common.model.WorkflowContext -import edu.uci.ics.texera.web.model.websocket.event.{ - TexeraWebSocketEvent, - WorkflowErrorEvent, - WorkflowStateEvent -} +import edu.uci.ics.texera.web.model.websocket.event.{TexeraWebSocketEvent, WorkflowErrorEvent, WorkflowStateEvent} import edu.uci.ics.texera.web.model.websocket.request.WorkflowExecuteRequest import edu.uci.ics.texera.web.storage.ExecutionStateStore import edu.uci.ics.texera.web.storage.ExecutionStateStore.updateWorkflowState -import edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore +import edu.uci.ics.amber.engine.common.ExecutionMetadataStore +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState.{COMPLETED, FAILED, READY} import edu.uci.ics.texera.web.{SubscriptionManager, TexeraWebApplication, WebsocketInput} import edu.uci.ics.texera.workflow.common.workflow.{LogicalPlan, WorkflowCompiler} @@ -119,7 +115,7 @@ class WorkflowExecutionService( .withFatalErrors(Seq.empty) ) executionStateStore.statsStore.updateState(stats => - stats.withStartTimeStamp(System.currentTimeMillis()) + stats.withStartTimestamp(System.currentTimeMillis()) ) client.controllerInterface .startWorkflow(EmptyRequest(), ()) diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowService.scala index f8aca483529..b971016e0dc 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowService.scala @@ -3,22 +3,11 @@ package edu.uci.ics.texera.web.service import com.google.protobuf.timestamp.Timestamp import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.architecture.controller.ControllerConfig -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.{ - COMPLETED, - FAILED -} -import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.{ - FaultToleranceConfig, - StateRestoreConfig -} +import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.{FaultToleranceConfig, StateRestoreConfig} import edu.uci.ics.amber.engine.common.AmberConfig import edu.uci.ics.amber.engine.common.amberexception.WorkflowRuntimeException import edu.uci.ics.amber.engine.common.model.WorkflowContext -import edu.uci.ics.amber.engine.common.virtualidentity.{ - ChannelMarkerIdentity, - ExecutionIdentity, - WorkflowIdentity -} +import edu.uci.ics.amber.engine.common.{ChannelMarkerIdentity, ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.amber.error.ErrorUtils.{getOperatorFromActorIdOpt, getStackTraceWithAllCauses} import edu.uci.ics.texera.web.model.jooq.generated.tables.pojos.User import edu.uci.ics.texera.web.model.websocket.event.TexeraWebSocketEvent @@ -26,8 +15,9 @@ import edu.uci.ics.texera.web.model.websocket.request.WorkflowExecuteRequest import edu.uci.ics.texera.web.service.WorkflowService.mkWorkflowStateId import edu.uci.ics.texera.web.storage.ExecutionStateStore.updateWorkflowState import edu.uci.ics.texera.web.storage.{ExecutionStateStore, WorkflowStateStore} -import edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.EXECUTION_FAILURE -import edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError +import edu.uci.ics.amber.engine.common.FatalErrorType.EXECUTION_FAILURE +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState._ +import edu.uci.ics.amber.engine.common.WorkflowFatalError import edu.uci.ics.texera.web.{SubscriptionManager, WorkflowLifecycleManager} import edu.uci.ics.texera.workflow.common.storage.OpResultStorage import edu.uci.ics.texera.workflow.common.workflow.LogicalPlan @@ -212,7 +202,7 @@ class WorkflowService( val (operatorId, workerId) = getOperatorFromActorIdOpt(fromActorOpt) logger.error("error during execution", t) executionStateStore.statsStore.updateState(stats => - stats.withEndTimeStamp(System.currentTimeMillis()) + stats.withEndTimestamp(System.currentTimeMillis()) ) executionStateStore.metadataStore.updateState { metadataStore => updateWorkflowState(FAILED, metadataStore).addFatalErrors( diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/ExecutionReconfigurationStore.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/ExecutionReconfigurationStore.scala index ecc501fc6cc..e0c8e80ea2e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/ExecutionReconfigurationStore.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/ExecutionReconfigurationStore.scala @@ -1,7 +1,7 @@ package edu.uci.ics.texera.web.storage import edu.uci.ics.amber.engine.common.model.PhysicalOp -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity import edu.uci.ics.texera.workflow.common.operators.StateTransferFunc case class ExecutionReconfigurationStore( diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/ExecutionStateStore.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/ExecutionStateStore.scala index de4a76363bb..484db4865ab 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/ExecutionStateStore.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/ExecutionStateStore.scala @@ -1,14 +1,8 @@ package edu.uci.ics.texera.web.storage -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState import edu.uci.ics.amber.engine.common.Utils.maptoStatusCode import edu.uci.ics.texera.web.service.ExecutionsMetadataPersistService -import edu.uci.ics.amber.engine.common.workflowruntimestate.{ - ExecutionBreakpointStore, - ExecutionConsoleStore, - ExecutionMetadataStore, - ExecutionStatsStore -} +import edu.uci.ics.amber.engine.common.{ExecutionBreakpointStore, ExecutionConsoleStore, ExecutionMetadataStore, ExecutionStatsStore, WorkflowAggregatedState} import java.sql.Timestamp diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/WorkflowResultStore.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/WorkflowResultStore.scala index bdcb40e81fc..3386eb372f6 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/WorkflowResultStore.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/WorkflowResultStore.scala @@ -1,6 +1,6 @@ package edu.uci.ics.texera.web.storage -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity +import edu.uci.ics.amber.engine.common.OperatorIdentity case class OperatorResultMetadata(tupleCount: Int = 0, changeDetector: String = "") diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/metadata/OperatorMetadataGenerator.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/metadata/OperatorMetadataGenerator.scala index 91923a28cfa..406cee29182 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/metadata/OperatorMetadataGenerator.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/metadata/OperatorMetadataGenerator.scala @@ -8,7 +8,7 @@ import com.kjetland.jackson.jsonSchema.JsonSchemaConfig.html5EnabledSchema import com.kjetland.jackson.jsonSchema.{JsonSchemaConfig, JsonSchemaDraft, JsonSchemaGenerator} import edu.uci.ics.amber.engine.common.Utils.objectMapper import edu.uci.ics.texera.workflow.common.operators.LogicalOp -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.source.scan.csv.CSVScanSourceOpDesc import java.util diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/LogicalOp.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/LogicalOp.scala index be68212a96f..675f985e121 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/LogicalOp.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/LogicalOp.scala @@ -12,12 +12,12 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.executor.OperatorExecutor import edu.uci.ics.amber.engine.common.model.{PhysicalOp, PhysicalPlan, WorkflowContext} import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.common.{ ExecutionIdentity, OperatorIdentity, WorkflowIdentity } -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity import edu.uci.ics.texera.web.OPversion import edu.uci.ics.texera.workflow.common.metadata.{OperatorInfo, PropertyNameConstants} import edu.uci.ics.texera.workflow.operators.aggregate.AggregateOpDesc diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/PythonOperatorDescriptor.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/PythonOperatorDescriptor.scala index 689652e144c..85216407cd1 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/PythonOperatorDescriptor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/PythonOperatorDescriptor.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.workflow.common.operators import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfoWithCode import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} trait PythonOperatorDescriptor extends LogicalOp { override def getPhysicalOp( diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/filter/FilterOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/filter/FilterOpDesc.scala index 7b14f7386a8..2c3b9e91b2a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/filter/FilterOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/filter/FilterOpDesc.scala @@ -3,7 +3,7 @@ package edu.uci.ics.texera.workflow.common.operators.filter import com.google.common.base.Preconditions import edu.uci.ics.amber.engine.common.model.PhysicalOp import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.workflow.common.operators.{LogicalOp, StateTransferFunc} import scala.util.{Success, Try} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/map/MapOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/map/MapOpDesc.scala index 538e00f0875..121e8b918a7 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/map/MapOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/map/MapOpDesc.scala @@ -1,7 +1,7 @@ package edu.uci.ics.texera.workflow.common.operators.map import edu.uci.ics.amber.engine.common.model.PhysicalOp -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.workflow.common.operators.{LogicalOp, StateTransferFunc} import scala.util.{Failure, Success, Try} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/storage/OpResultStorage.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/storage/OpResultStorage.scala index e22179c5189..92eed786921 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/storage/OpResultStorage.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/storage/OpResultStorage.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.workflow.common.storage import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.common.AmberConfig -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity +import edu.uci.ics.amber.engine.common.OperatorIdentity import edu.uci.ics.texera.workflow.operators.sink.storage.{ MemoryStorage, MongoDBSinkStorage, diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/LogicalLink.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/LogicalLink.scala index 91e87c78d2d..f1695141326 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/LogicalLink.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/LogicalLink.scala @@ -1,8 +1,8 @@ package edu.uci.ics.texera.workflow.common.workflow import com.fasterxml.jackson.annotation.{JsonCreator, JsonProperty} -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.OperatorIdentity +import edu.uci.ics.amber.engine.common.PortIdentity case class LogicalLink( @JsonProperty("fromOpId") fromOpId: OperatorIdentity, diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/LogicalPlan.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/LogicalPlan.scala index 1574f1cd373..a6c9782f1c2 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/LogicalPlan.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/LogicalPlan.scala @@ -3,8 +3,8 @@ package edu.uci.ics.texera.workflow.common.workflow import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.common.model.WorkflowContext import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.OperatorIdentity +import edu.uci.ics.amber.engine.common.PortIdentity import edu.uci.ics.texera.web.model.websocket.request.LogicalPlanPojo import edu.uci.ics.texera.workflow.common.operators.LogicalOp import edu.uci.ics.texera.workflow.common.operators.source.SourceOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/LogicalPort.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/LogicalPort.scala index fe64378e031..0838f3ed5d6 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/LogicalPort.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/LogicalPort.scala @@ -1,6 +1,6 @@ package edu.uci.ics.texera.workflow.common.workflow -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity +import edu.uci.ics.amber.engine.common.OperatorIdentity case object LogicalPort { def apply(operatorIdentity: OperatorIdentity, portOrdinal: Integer): LogicalPort = { diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/SinkInjectionTransformer.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/SinkInjectionTransformer.scala index 1e34daafac1..c8b22c6a359 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/SinkInjectionTransformer.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/SinkInjectionTransformer.scala @@ -1,7 +1,7 @@ package edu.uci.ics.texera.workflow.common.workflow -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.OperatorIdentity +import edu.uci.ics.amber.engine.common.PortIdentity import edu.uci.ics.texera.workflow.operators.sink.SinkOpDesc import edu.uci.ics.texera.workflow.operators.sink.managed.ProgressiveSinkOpDesc import edu.uci.ics.texera.workflow.operators.visualization.VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/WorkflowCompiler.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/WorkflowCompiler.scala index 67510946979..55a2d97298c 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/WorkflowCompiler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/workflow/WorkflowCompiler.scala @@ -3,18 +3,18 @@ package edu.uci.ics.texera.workflow.common.workflow import com.google.protobuf.timestamp.Timestamp import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.architecture.controller.Workflow -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.FAILED import edu.uci.ics.amber.engine.common.model.{PhysicalPlan, WorkflowContext} import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity +import edu.uci.ics.amber.engine.common.OperatorIdentity import edu.uci.ics.amber.error.ErrorUtils.getStackTraceWithAllCauses import edu.uci.ics.amber.engine.common.Utils.objectMapper import edu.uci.ics.texera.web.model.websocket.request.LogicalPlanPojo import edu.uci.ics.texera.web.service.ExecutionsMetadataPersistService import edu.uci.ics.texera.web.storage.ExecutionStateStore import edu.uci.ics.texera.web.storage.ExecutionStateStore.updateWorkflowState -import edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.COMPILATION_ERROR -import edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError +import edu.uci.ics.amber.engine.common.FatalErrorType.COMPILATION_ERROR +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState.FAILED +import edu.uci.ics.amber.engine.common.WorkflowFatalError import edu.uci.ics.texera.workflow.common.storage.OpResultStorage import edu.uci.ics.texera.workflow.operators.sink.managed.ProgressiveSinkOpDesc import edu.uci.ics.texera.workflow.operators.visualization.VisualizationConstants diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/aggregate/AggregateOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/aggregate/AggregateOpDesc.scala index 63cb5ced6e8..ea1bf8a5a7b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/aggregate/AggregateOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/aggregate/AggregateOpDesc.scala @@ -5,12 +5,12 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, PhysicalPlan, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.common.{ ExecutionIdentity, PhysicalOpIdentity, WorkflowIdentity } -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PhysicalLink, PortIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PhysicalLink, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeNameList import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.LogicalOp diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/cartesianProduct/CartesianProductOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/cartesianProduct/CartesianProductOpDesc.scala index 1cee7475e69..6a4c9685ab7 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/cartesianProduct/CartesianProductOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/cartesianProduct/CartesianProductOpDesc.scala @@ -3,8 +3,8 @@ package edu.uci.ics.texera.workflow.operators.cartesianProduct import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.LogicalOp diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/dictionary/DictionaryMatcherOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/dictionary/DictionaryMatcherOpDesc.scala index df0c2a6651d..801e4fdc365 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/dictionary/DictionaryMatcherOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/dictionary/DictionaryMatcherOpDesc.scala @@ -5,8 +5,8 @@ import com.google.common.base.Preconditions import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.map.MapOpDesc diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/difference/DifferenceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/difference/DifferenceOpDesc.scala index afe5455a3db..7487e2337c0 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/difference/DifferenceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/difference/DifferenceOpDesc.scala @@ -4,8 +4,8 @@ import com.google.common.base.Preconditions import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.PhysicalOp import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.LogicalOp import edu.uci.ics.texera.workflow.common.workflow.HashPartition diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/distinct/DistinctOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/distinct/DistinctOpDesc.scala index 8c0548e7510..1ad28d2ecc6 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/distinct/DistinctOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/distinct/DistinctOpDesc.scala @@ -4,8 +4,8 @@ import com.google.common.base.Preconditions import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.PhysicalOp import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.LogicalOp import edu.uci.ics.texera.workflow.common.workflow.HashPartition diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/download/BulkDownloaderOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/download/BulkDownloaderOpDesc.scala index c55bbc7916f..4cc1b650500 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/download/BulkDownloaderOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/download/BulkDownloaderOpDesc.scala @@ -6,8 +6,8 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.LogicalOp diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/dummy/DummyOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/dummy/DummyOpDesc.scala index 65325c6d750..324bc2da8e7 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/dummy/DummyOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/dummy/DummyOpDesc.scala @@ -5,7 +5,7 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.model.tuple.Schema import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PortDescription -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.operators.{LogicalOp, PortDescriptor} class DummyOpDesc extends LogicalOp with PortDescriptor { diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/filter/SpecializedFilterOpDesc.java b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/filter/SpecializedFilterOpDesc.java index f6d7066bb19..715760fcb23 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/filter/SpecializedFilterOpDesc.java +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/filter/SpecializedFilterOpDesc.java @@ -4,11 +4,11 @@ import com.fasterxml.jackson.annotation.JsonPropertyDescription; import edu.uci.ics.amber.engine.common.model.PhysicalOp; import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo; -import edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity; -import edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity; -import edu.uci.ics.amber.engine.common.workflow.InputPort; -import edu.uci.ics.amber.engine.common.workflow.OutputPort; -import edu.uci.ics.amber.engine.common.workflow.PortIdentity; +import edu.uci.ics.amber.engine.common.ExecutionIdentity; +import edu.uci.ics.amber.engine.common.WorkflowIdentity; +import edu.uci.ics.amber.engine.common.InputPort; +import edu.uci.ics.amber.engine.common.OutputPort; +import edu.uci.ics.amber.engine.common.PortIdentity; import edu.uci.ics.texera.workflow.common.metadata.OperatorGroupConstants; import edu.uci.ics.texera.workflow.common.metadata.OperatorInfo; import edu.uci.ics.amber.engine.common.executor.OperatorExecutor; diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/hashJoin/HashJoinOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/hashJoin/HashJoinOpDesc.scala index 4ccd5a66d0e..635735c1fb5 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/hashJoin/HashJoinOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/hashJoin/HashJoinOpDesc.scala @@ -6,12 +6,12 @@ import com.kjetland.jackson.jsonSchema.annotations.{JsonSchemaInject, JsonSchema import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, PhysicalPlan, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.common.{ ExecutionIdentity, PhysicalOpIdentity, WorkflowIdentity } -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PhysicalLink, PortIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PhysicalLink, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.{ AutofillAttributeName, AutofillAttributeNameOnPort1 diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceIrisLogisticRegressionOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceIrisLogisticRegressionOpDesc.scala index 271633d52ff..79efb4cbcb0 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceIrisLogisticRegressionOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceIrisLogisticRegressionOpDesc.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.workflow.operators.huggingFace import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceSentimentAnalysisOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceSentimentAnalysisOpDesc.scala index 3ae360b5e89..acc69e182e3 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceSentimentAnalysisOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceSentimentAnalysisOpDesc.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.workflow.operators.huggingFace import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceSpamSMSDetectionOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceSpamSMSDetectionOpDesc.scala index 1b35c5a0183..525d29584d5 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceSpamSMSDetectionOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceSpamSMSDetectionOpDesc.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.workflow.operators.huggingFace import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceTextSummarizationOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceTextSummarizationOpDesc.scala index 523bab2beff..b955192c268 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceTextSummarizationOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/huggingFace/HuggingFaceTextSummarizationOpDesc.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.workflow.operators.huggingFace import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/intersect/IntersectOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/intersect/IntersectOpDesc.scala index fc599059f07..0ed5f98379d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/intersect/IntersectOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/intersect/IntersectOpDesc.scala @@ -4,8 +4,8 @@ import com.google.common.base.Preconditions import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.PhysicalOp import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.LogicalOp import edu.uci.ics.texera.workflow.common.workflow.HashPartition diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/intervalJoin/IntervalJoinOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/intervalJoin/IntervalJoinOpDesc.scala index 987dc2aac44..13063444862 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/intervalJoin/IntervalJoinOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/intervalJoin/IntervalJoinOpDesc.scala @@ -6,8 +6,8 @@ import com.kjetland.jackson.jsonSchema.annotations.{JsonSchemaInject, JsonSchema import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.{ AutofillAttributeName, AutofillAttributeNameOnPort1 diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/keywordSearch/KeywordSearchOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/keywordSearch/KeywordSearchOpDesc.scala index 505368fd9ba..ff2bc033b36 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/keywordSearch/KeywordSearchOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/keywordSearch/KeywordSearchOpDesc.scala @@ -4,8 +4,8 @@ import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.PhysicalOp -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.filter.FilterOpDesc diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/limit/LimitOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/limit/LimitOpDesc.scala index 93193c88da0..263b3962314 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/limit/LimitOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/limit/LimitOpDesc.scala @@ -5,8 +5,8 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.PhysicalOp import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.{LogicalOp, StateTransferFunc} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/machineLearning/Scorer/MachineLearningScorerOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/machineLearning/Scorer/MachineLearningScorerOpDesc.scala index 6f5fe755d26..e8421f2e1fc 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/machineLearning/Scorer/MachineLearningScorerOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/machineLearning/Scorer/MachineLearningScorerOpDesc.scala @@ -7,7 +7,7 @@ import com.kjetland.jackson.jsonSchema.annotations.{ JsonSchemaTitle } import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.annotations.{ AutofillAttributeName, HideAnnotation diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/machineLearning/sklearnAdvanced/base/SklearnAdvancedBaseDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/machineLearning/sklearnAdvanced/base/SklearnAdvancedBaseDesc.scala index dc8535c0a5a..adb3b4e3219 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/machineLearning/sklearnAdvanced/base/SklearnAdvancedBaseDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/machineLearning/sklearnAdvanced/base/SklearnAdvancedBaseDesc.scala @@ -3,7 +3,7 @@ package edu.uci.ics.texera.workflow.operators.machineLearning.sklearnAdvanced.ba import com.fasterxml.jackson.annotation.{JsonIgnore, JsonProperty, JsonPropertyDescription} import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.{ AutofillAttributeName, AutofillAttributeNameList diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/projection/ProjectionOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/projection/ProjectionOpDesc.scala index 6f6118b9d9a..d8e8c25842d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/projection/ProjectionOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/projection/ProjectionOpDesc.scala @@ -7,8 +7,8 @@ import edu.uci.ics.amber.engine.common.model.PhysicalOp.oneToOnePhysicalOp import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata._ import edu.uci.ics.texera.workflow.common.operators.map.MapOpDesc import edu.uci.ics.texera.workflow.common.workflow.{ diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/randomksampling/RandomKSamplingOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/randomksampling/RandomKSamplingOpDesc.scala index 043d0845fc8..fd8b98df43f 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/randomksampling/RandomKSamplingOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/randomksampling/RandomKSamplingOpDesc.scala @@ -4,8 +4,8 @@ import com.fasterxml.jackson.annotation.{JsonIgnore, JsonProperty, JsonPropertyD import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.AmberConfig import edu.uci.ics.amber.engine.common.model.PhysicalOp -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.filter.FilterOpDesc diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/regex/RegexOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/regex/RegexOpDesc.scala index 8cdb421c609..555fba330f5 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/regex/RegexOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/regex/RegexOpDesc.scala @@ -4,8 +4,8 @@ import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.PhysicalOp -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.operators.filter.FilterOpDesc diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/reservoirsampling/ReservoirSamplingOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/reservoirsampling/ReservoirSamplingOpDesc.scala index 226de2aa956..c2e6c996949 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/reservoirsampling/ReservoirSamplingOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/reservoirsampling/ReservoirSamplingOpDesc.scala @@ -6,8 +6,8 @@ import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInf import edu.uci.ics.amber.engine.common.AmberConfig import edu.uci.ics.amber.engine.common.model.PhysicalOp import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.LogicalOp import edu.uci.ics.texera.workflow.operators.util.OperatorDescriptorUtils.equallyPartitionGoal diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sentiment/SentimentAnalysisOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sentiment/SentimentAnalysisOpDesc.scala index b69b5dc0515..0ae821ec85f 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sentiment/SentimentAnalysisOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sentiment/SentimentAnalysisOpDesc.scala @@ -5,8 +5,8 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaInject import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.operators.map.MapOpDesc diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sink/managed/ProgressiveSinkOpDesc.java b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sink/managed/ProgressiveSinkOpDesc.java index c630c62560c..32d6801e1b9 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sink/managed/ProgressiveSinkOpDesc.java +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sink/managed/ProgressiveSinkOpDesc.java @@ -5,12 +5,12 @@ import edu.uci.ics.amber.engine.common.model.PhysicalOp; import edu.uci.ics.amber.engine.common.model.SchemaPropagationFunc; import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo; -import edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity; -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity; -import edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity; -import edu.uci.ics.amber.engine.common.workflow.InputPort; -import edu.uci.ics.amber.engine.common.workflow.OutputPort; -import edu.uci.ics.amber.engine.common.workflow.PortIdentity; +import edu.uci.ics.amber.engine.common.ExecutionIdentity; +import edu.uci.ics.amber.engine.common.OperatorIdentity; +import edu.uci.ics.amber.engine.common.WorkflowIdentity; +import edu.uci.ics.amber.engine.common.InputPort; +import edu.uci.ics.amber.engine.common.OutputPort; +import edu.uci.ics.amber.engine.common.PortIdentity; import edu.uci.ics.amber.engine.common.IncrementalOutputMode; import edu.uci.ics.amber.engine.common.ProgressiveUtils; import edu.uci.ics.texera.workflow.common.metadata.OperatorGroupConstants; diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sklearn/SklearnMLOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sklearn/SklearnMLOpDesc.scala index a46ce9dc371..3062db2c7ed 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sklearn/SklearnMLOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sklearn/SklearnMLOpDesc.scala @@ -8,7 +8,7 @@ import com.kjetland.jackson.jsonSchema.annotations.{ JsonSchemaTitle } import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.{ AutofillAttributeName, CommonOpDescAnnotation, diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sklearn/SklearnPredictionOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sklearn/SklearnPredictionOpDesc.scala index 2ae3b50d78f..f8c734d2aaf 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sklearn/SklearnPredictionOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sklearn/SklearnPredictionOpDesc.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.workflow.operators.sklearn import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sort/SortOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sort/SortOpDesc.scala index 1727160ac90..2aae228cea2 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sort/SortOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sort/SortOpDesc.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.workflow.operators.sort import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sortPartitions/SortPartitionsOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sortPartitions/SortPartitionsOpDesc.scala index ce090701fed..b94da06a8d4 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sortPartitions/SortPartitionsOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/sortPartitions/SortPartitionsOpDesc.scala @@ -6,8 +6,8 @@ import com.kjetland.jackson.jsonSchema.annotations.{JsonSchemaInject, JsonSchema import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.PhysicalOp import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.LogicalOp diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/reddit/RedditSearchSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/reddit/RedditSearchSourceOpDesc.scala index c29cb58d95d..062692d6a69 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/reddit/RedditSearchSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/reddit/RedditSearchSourceOpDesc.scala @@ -4,7 +4,7 @@ import com.fasterxml.jackson.annotation.JsonProperty import com.fasterxml.jackson.annotation.JsonPropertyDescription import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.OutputPort +import edu.uci.ics.amber.engine.common.OutputPort import edu.uci.ics.texera.workflow.common.metadata.OperatorGroupConstants import edu.uci.ics.texera.workflow.common.metadata.OperatorInfo import edu.uci.ics.texera.workflow.common.operators.source.PythonSourceOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/twitter/TwitterSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/twitter/TwitterSourceOpDesc.scala index 5777b445c56..78160bbf32e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/twitter/TwitterSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/twitter/TwitterSourceOpDesc.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.workflow.operators.source.apis.twitter import com.fasterxml.jackson.annotation.{JsonIgnore, JsonProperty} import com.kjetland.jackson.jsonSchema.annotations.{JsonSchemaDescription, JsonSchemaTitle} -import edu.uci.ics.amber.engine.common.workflow.OutputPort +import edu.uci.ics.amber.engine.common.OutputPort import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.source.SourceOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/twitter/v2/TwitterFullArchiveSearchSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/twitter/v2/TwitterFullArchiveSearchSourceOpDesc.scala index 4af4885c8d7..60a7e872c27 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/twitter/v2/TwitterFullArchiveSearchSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/twitter/v2/TwitterFullArchiveSearchSourceOpDesc.scala @@ -9,7 +9,7 @@ import com.kjetland.jackson.jsonSchema.annotations.{ import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.UIWidget import edu.uci.ics.texera.workflow.operators.source.apis.twitter.TwitterSourceOpDesc diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/twitter/v2/TwitterSearchSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/twitter/v2/TwitterSearchSourceOpDesc.scala index 0be04b6e030..534f067df93 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/twitter/v2/TwitterSearchSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/apis/twitter/v2/TwitterSearchSourceOpDesc.scala @@ -9,7 +9,7 @@ import com.kjetland.jackson.jsonSchema.annotations.{ import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.UIWidget import edu.uci.ics.texera.workflow.operators.source.apis.twitter.TwitterSourceOpDesc diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/cache/CacheSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/cache/CacheSourceOpDesc.scala index 8a60c0bab30..738eebef87d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/cache/CacheSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/cache/CacheSourceOpDesc.scala @@ -3,12 +3,12 @@ package edu.uci.ics.texera.workflow.operators.source.cache import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.common.{ ExecutionIdentity, OperatorIdentity, WorkflowIdentity } -import edu.uci.ics.amber.engine.common.workflow.OutputPort +import edu.uci.ics.amber.engine.common.OutputPort import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.source.SourceOperatorDescriptor import edu.uci.ics.texera.workflow.common.storage.OpResultStorage diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/fetcher/URLFetcherOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/fetcher/URLFetcherOpDesc.scala index b3d1cd01c8e..58bcf0e9cb0 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/fetcher/URLFetcherOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/fetcher/URLFetcherOpDesc.scala @@ -5,8 +5,8 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.OutputPort +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.OutputPort import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.source.SourceOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/FileScanSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/FileScanSourceOpDesc.scala index ee1621e762f..f13bebcac85 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/FileScanSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/FileScanSourceOpDesc.scala @@ -9,7 +9,7 @@ import com.kjetland.jackson.jsonSchema.annotations.{ import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.HideAnnotation import edu.uci.ics.texera.workflow.operators.source.scan.text.TextSourceOpDesc diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/ScanSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/ScanSourceOpDesc.scala index 46ff9196ef0..9d5c4766c23 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/ScanSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/ScanSourceOpDesc.scala @@ -5,7 +5,7 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.model.WorkflowContext import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.workflow.OutputPort +import edu.uci.ics.amber.engine.common.OutputPort import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.source.SourceOperatorDescriptor import org.apache.commons.lang3.builder.EqualsBuilder diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/CSVScanSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/CSVScanSourceOpDesc.scala index 5e5ccdfd26b..d2087deb93c 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/CSVScanSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/CSVScanSourceOpDesc.scala @@ -6,7 +6,7 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import com.univocity.parsers.csv.{CsvFormat, CsvParser, CsvParserSettings} import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.amber.engine.common.model.tuple.AttributeTypeUtils.inferSchemaFromRows import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} import edu.uci.ics.amber.engine.common.storage.DocumentFactory diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/CSVScanSourceOpExec.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/CSVScanSourceOpExec.scala index 245e67e1dfe..4b856d06282 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/CSVScanSourceOpExec.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/CSVScanSourceOpExec.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.workflow.operators.source.scan.csv import com.univocity.parsers.csv.{CsvFormat, CsvParser, CsvParserSettings} import edu.uci.ics.amber.engine.common.executor.SourceOperatorExecutor -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity import edu.uci.ics.amber.engine.common.{CheckpointState, CheckpointSupport} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeTypeUtils, Schema, TupleLike} import edu.uci.ics.amber.engine.common.storage.DocumentFactory diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/ParallelCSVScanSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/ParallelCSVScanSourceOpDesc.scala index 2978cb56992..cec470764ce 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/ParallelCSVScanSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/ParallelCSVScanSourceOpDesc.scala @@ -6,7 +6,7 @@ import com.github.tototoshi.csv.{CSVReader, DefaultCSVFormat} import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.amber.engine.common.model.tuple.AttributeTypeUtils.inferSchemaFromRows import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} import edu.uci.ics.amber.engine.common.storage.DocumentFactory diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csvOld/CSVOldScanSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csvOld/CSVOldScanSourceOpDesc.scala index 38625b62b0f..88c2521d4dd 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csvOld/CSVOldScanSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/csvOld/CSVOldScanSourceOpDesc.scala @@ -6,7 +6,7 @@ import com.github.tototoshi.csv.{CSVReader, DefaultCSVFormat} import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.amber.engine.common.model.tuple.AttributeTypeUtils.inferSchemaFromRows import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} import edu.uci.ics.amber.engine.common.storage.DocumentFactory diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/json/JSONLScanSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/json/JSONLScanSourceOpDesc.scala index c622a6e8853..968566eae41 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/json/JSONLScanSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/json/JSONLScanSourceOpDesc.scala @@ -5,7 +5,7 @@ import com.fasterxml.jackson.databind.JsonNode import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.storage.{DatasetFileDocument, DocumentFactory} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.amber.engine.common.Utils.objectMapper import edu.uci.ics.amber.engine.common.model.tuple.AttributeTypeUtils.inferSchemaFromRows import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, Schema} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/text/TextInputSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/text/TextInputSourceOpDesc.scala index 546faea3abc..2b43d383d26 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/text/TextInputSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/scan/text/TextInputSourceOpDesc.scala @@ -5,8 +5,8 @@ import com.kjetland.jackson.jsonSchema.annotations.{JsonSchemaInject, JsonSchema import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.OutputPort +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.OutputPort import edu.uci.ics.texera.workflow.common.metadata.annotations.UIWidget import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.source.SourceOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/sql/asterixdb/AsterixDBSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/sql/asterixdb/AsterixDBSourceOpDesc.scala index 03e690889d9..1953c1504e7 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/sql/asterixdb/AsterixDBSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/sql/asterixdb/AsterixDBSourceOpDesc.scala @@ -10,8 +10,8 @@ import com.kjetland.jackson.jsonSchema.annotations.{JsonSchemaInject, JsonSchema import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.OutputPort +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.OutputPort import edu.uci.ics.texera.workflow.common.metadata.annotations.{ AutofillAttributeName, AutofillAttributeNameList, diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/sql/mysql/MySQLSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/sql/mysql/MySQLSourceOpDesc.scala index a4fdb25e0bf..da56e6d146d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/sql/mysql/MySQLSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/sql/mysql/MySQLSourceOpDesc.scala @@ -2,8 +2,8 @@ package edu.uci.ics.texera.workflow.operators.source.sql.mysql import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.OutputPort +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.OutputPort import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.operators.source.sql.SQLSourceOpDesc import edu.uci.ics.texera.workflow.operators.source.sql.mysql.MySQLConnUtil.connect diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/sql/postgresql/PostgreSQLSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/sql/postgresql/PostgreSQLSourceOpDesc.scala index 3693f627a51..a498c8bd768 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/sql/postgresql/PostgreSQLSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/sql/postgresql/PostgreSQLSourceOpDesc.scala @@ -5,8 +5,8 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize import com.kjetland.jackson.jsonSchema.annotations.{JsonSchemaInject, JsonSchemaTitle} import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.OutputPort +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.OutputPort import edu.uci.ics.texera.workflow.common.metadata.annotations.UIWidget import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.operators.source.sql.SQLSourceOpDesc diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/split/SplitOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/split/SplitOpDesc.scala index ffa9dfab0d3..4feda266b43 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/split/SplitOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/split/SplitOpDesc.scala @@ -5,8 +5,8 @@ import com.google.common.base.Preconditions import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.LogicalOp diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/split/SplitOpExec.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/split/SplitOpExec.scala index 2fefd46f846..a38adda247e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/split/SplitOpExec.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/split/SplitOpExec.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.workflow.operators.split import edu.uci.ics.amber.engine.common.executor.OperatorExecutor import edu.uci.ics.amber.engine.common.model.tuple.{Tuple, TupleLike} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity import scala.util.Random diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/symmetricDifference/SymmetricDifferenceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/symmetricDifference/SymmetricDifferenceOpDesc.scala index 702b8773d46..cb406a1ce36 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/symmetricDifference/SymmetricDifferenceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/symmetricDifference/SymmetricDifferenceOpDesc.scala @@ -4,10 +4,10 @@ import com.google.common.base.Preconditions import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.PhysicalOp import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.LogicalOp -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.workflow.HashPartition class SymmetricDifferenceOpDesc extends LogicalOp { diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/typecasting/TypeCastingOpDesc.java b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/typecasting/TypeCastingOpDesc.java index 65250f19562..2755dcdd9ec 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/typecasting/TypeCastingOpDesc.java +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/typecasting/TypeCastingOpDesc.java @@ -7,11 +7,11 @@ import edu.uci.ics.amber.engine.common.model.SchemaPropagationFunc; import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo; import edu.uci.ics.amber.engine.common.model.tuple.AttributeTypeUtils; -import edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity; -import edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity; -import edu.uci.ics.amber.engine.common.workflow.InputPort; -import edu.uci.ics.amber.engine.common.workflow.OutputPort; -import edu.uci.ics.amber.engine.common.workflow.PortIdentity; +import edu.uci.ics.amber.engine.common.ExecutionIdentity; +import edu.uci.ics.amber.engine.common.WorkflowIdentity; +import edu.uci.ics.amber.engine.common.InputPort; +import edu.uci.ics.amber.engine.common.OutputPort; +import edu.uci.ics.amber.engine.common.PortIdentity; import edu.uci.ics.texera.workflow.common.metadata.OperatorGroupConstants; import edu.uci.ics.texera.workflow.common.metadata.OperatorInfo; import edu.uci.ics.amber.engine.common.executor.OperatorExecutor; diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/java/JavaUDFOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/java/JavaUDFOpDesc.scala index ec51f183c06..e37fce01eb2 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/java/JavaUDFOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/java/JavaUDFOpDesc.scala @@ -6,8 +6,8 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.{LogicalOp, PortDescription, StateTransferFunc} import edu.uci.ics.texera.workflow.common.workflow.{PartitionInfo, UnknownPartition} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/DualInputPortsPythonUDFOpDescV2.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/DualInputPortsPythonUDFOpDescV2.scala index c574d5df281..d8d2ab004d3 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/DualInputPortsPythonUDFOpDescV2.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/DualInputPortsPythonUDFOpDescV2.scala @@ -6,13 +6,13 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.LogicalOp import edu.uci.ics.texera.workflow.common.workflow.UnknownPartition -import edu.uci.ics.amber.engine.common.workflow.InputPort -import edu.uci.ics.amber.engine.common.workflow.OutputPort -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.InputPort +import edu.uci.ics.amber.engine.common.OutputPort +import edu.uci.ics.amber.engine.common.PortIdentity class DualInputPortsPythonUDFOpDescV2 extends LogicalOp { @JsonProperty( diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/PythonLambdaFunctionOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/PythonLambdaFunctionOpDesc.scala index 7c931cd9b64..c25c15d8345 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/PythonLambdaFunctionOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/PythonLambdaFunctionOpDesc.scala @@ -3,7 +3,7 @@ package edu.uci.ics.texera.workflow.operators.udf.python import com.google.common.base.Preconditions import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.model.tuple.{AttributeTypeUtils, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/PythonTableReducerOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/PythonTableReducerOpDesc.scala index 5ab1b92a028..cb2de9ed7df 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/PythonTableReducerOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/PythonTableReducerOpDesc.scala @@ -3,7 +3,7 @@ package edu.uci.ics.texera.workflow.operators.udf.python import com.google.common.base.Preconditions import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/PythonUDFOpDescV2.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/PythonUDFOpDescV2.scala index 5ea14eb1540..42d84e591bd 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/PythonUDFOpDescV2.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/PythonUDFOpDescV2.scala @@ -6,11 +6,11 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.{LogicalOp, PortDescription, StateTransferFunc} import edu.uci.ics.texera.workflow.common.workflow.{PartitionInfo, UnknownPartition} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import scala.util.{Success, Try} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/source/PythonUDFSourceOpDescV2.java b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/source/PythonUDFSourceOpDescV2.java index 87e8b44a58b..673757d85dd 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/source/PythonUDFSourceOpDescV2.java +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/python/source/PythonUDFSourceOpDescV2.java @@ -7,11 +7,11 @@ import edu.uci.ics.amber.engine.common.model.PhysicalOp; import edu.uci.ics.amber.engine.common.model.SchemaPropagationFunc; import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo; -import edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity; -import edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity; -import edu.uci.ics.amber.engine.common.workflow.InputPort; -import edu.uci.ics.amber.engine.common.workflow.OutputPort; -import edu.uci.ics.amber.engine.common.workflow.PortIdentity; +import edu.uci.ics.amber.engine.common.ExecutionIdentity; +import edu.uci.ics.amber.engine.common.WorkflowIdentity; +import edu.uci.ics.amber.engine.common.InputPort; +import edu.uci.ics.amber.engine.common.OutputPort; +import edu.uci.ics.amber.engine.common.PortIdentity; import edu.uci.ics.texera.workflow.common.metadata.OperatorGroupConstants; import edu.uci.ics.texera.workflow.common.metadata.OperatorInfo; import edu.uci.ics.texera.workflow.common.operators.source.SourceOperatorDescriptor; diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/r/RUDFOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/r/RUDFOpDesc.scala index 4dc5f0ea8c8..170f1d2a765 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/r/RUDFOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/r/RUDFOpDesc.scala @@ -6,8 +6,8 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.{LogicalOp, PortDescription, StateTransferFunc} import edu.uci.ics.texera.workflow.common.workflow.{PartitionInfo, UnknownPartition} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/r/RUDFSourceOpDesc.java b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/r/RUDFSourceOpDesc.java index 7579774ec66..cf249c526d4 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/r/RUDFSourceOpDesc.java +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/udf/r/RUDFSourceOpDesc.java @@ -7,11 +7,11 @@ import edu.uci.ics.amber.engine.common.model.PhysicalOp; import edu.uci.ics.amber.engine.common.model.SchemaPropagationFunc; import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo; -import edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity; -import edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity; -import edu.uci.ics.amber.engine.common.workflow.InputPort; -import edu.uci.ics.amber.engine.common.workflow.OutputPort; -import edu.uci.ics.amber.engine.common.workflow.PortIdentity; +import edu.uci.ics.amber.engine.common.ExecutionIdentity; +import edu.uci.ics.amber.engine.common.WorkflowIdentity; +import edu.uci.ics.amber.engine.common.InputPort; +import edu.uci.ics.amber.engine.common.OutputPort; +import edu.uci.ics.amber.engine.common.PortIdentity; import edu.uci.ics.texera.workflow.common.metadata.OperatorGroupConstants; import edu.uci.ics.texera.workflow.common.metadata.OperatorInfo; import edu.uci.ics.texera.workflow.common.operators.source.SourceOperatorDescriptor; diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/union/UnionOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/union/UnionOpDesc.scala index d69f9403d5c..74c0c2b2003 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/union/UnionOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/union/UnionOpDesc.scala @@ -4,10 +4,10 @@ import com.google.common.base.Preconditions import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.PhysicalOp import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.LogicalOp -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} class UnionOpDesc extends LogicalOp { diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/unneststring/UnnestStringOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/unneststring/UnnestStringOpDesc.scala index 7177856f3dd..edc23081bdd 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/unneststring/UnnestStringOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/unneststring/UnnestStringOpDesc.scala @@ -5,11 +5,11 @@ import com.google.common.base.Preconditions import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.operators.flatmap.FlatMapOpDesc -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} class UnnestStringOpDesc extends FlatMapOpDesc { @JsonProperty(value = "Delimiter", required = true, defaultValue = ",") diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/DotPlot/DotPlotOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/DotPlot/DotPlotOpDesc.scala index 529c411226e..f0bee953016 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/DotPlot/DotPlotOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/DotPlot/DotPlotOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/IcicleChart/IcicleChartOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/IcicleChart/IcicleChartOpDesc.scala index 44d02861e3d..ed587b3e211 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/IcicleChart/IcicleChartOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/IcicleChart/IcicleChartOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ImageViz/ImageVisualizerOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ImageViz/ImageVisualizerOpDesc.scala index 5e0f3730bbd..1f65ef2079a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ImageViz/ImageVisualizerOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ImageViz/ImageVisualizerOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ScatterMatrixChart/ScatterMatrixChartOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ScatterMatrixChart/ScatterMatrixChartOpDesc.scala index ae4ec634c65..8dd56fa58e8 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ScatterMatrixChart/ScatterMatrixChartOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ScatterMatrixChart/ScatterMatrixChartOpDesc.scala @@ -9,7 +9,7 @@ import edu.uci.ics.texera.workflow.common.metadata.annotations.{ } import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/barChart/BarChartOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/barChart/BarChartOpDesc.scala index 4fca4913609..225fe446f88 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/barChart/BarChartOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/barChart/BarChartOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/boxPlot/BoxPlotOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/boxPlot/BoxPlotOpDesc.scala index 7f5b0ec2792..9d4ea36ba4a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/boxPlot/BoxPlotOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/boxPlot/BoxPlotOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/bubbleChart/BubbleChartOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/bubbleChart/BubbleChartOpDesc.scala index d6797aaccc8..1967c52c79d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/bubbleChart/BubbleChartOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/bubbleChart/BubbleChartOpDesc.scala @@ -7,7 +7,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/candlestickChart/CandlestickChartOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/candlestickChart/CandlestickChartOpDesc.scala index 79f2aeaf2aa..00a6deb5018 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/candlestickChart/CandlestickChartOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/candlestickChart/CandlestickChartOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/continuousErrorBands/ContinuousErrorBandsOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/continuousErrorBands/ContinuousErrorBandsOpDesc.scala index 1f1d562571d..24f2572c90a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/continuousErrorBands/ContinuousErrorBandsOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/continuousErrorBands/ContinuousErrorBandsOpDesc.scala @@ -5,7 +5,7 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/contourPlot/ContourPlotOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/contourPlot/ContourPlotOpDesc.scala index 2e21c2efb9a..d1c07ebead1 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/contourPlot/ContourPlotOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/contourPlot/ContourPlotOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/dumbbellPlot/DumbbellPlotOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/dumbbellPlot/DumbbellPlotOpDesc.scala index 921dc8688bc..d61a0e1c04a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/dumbbellPlot/DumbbellPlotOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/dumbbellPlot/DumbbellPlotOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/figureFactoryTable/FigureFactoryTableOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/figureFactoryTable/FigureFactoryTableOpDesc.scala index 37914b18586..5d38e662feb 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/figureFactoryTable/FigureFactoryTableOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/figureFactoryTable/FigureFactoryTableOpDesc.scala @@ -3,7 +3,7 @@ package edu.uci.ics.texera.workflow.operators.visualization.figureFactoryTable import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor import edu.uci.ics.texera.workflow.operators.visualization.{ diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/filledAreaPlot/FilledAreaPlotOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/filledAreaPlot/FilledAreaPlotOpDesc.scala index 9fd921a9c28..4392447a471 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/filledAreaPlot/FilledAreaPlotOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/filledAreaPlot/FilledAreaPlotOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/funnelPlot/FunnelPlotOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/funnelPlot/FunnelPlotOpDesc.scala index 4d7c291e186..90c254af2fc 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/funnelPlot/FunnelPlotOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/funnelPlot/FunnelPlotOpDesc.scala @@ -3,7 +3,7 @@ package edu.uci.ics.texera.workflow.operators.visualization.funnelPlot import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import com.kjetland.jackson.jsonSchema.annotations.{JsonSchemaInject, JsonSchemaTitle} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ganttChart/GanttChartOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ganttChart/GanttChartOpDesc.scala index 333248093d9..c23226fbe53 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ganttChart/GanttChartOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ganttChart/GanttChartOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/heatMap/HeatMapOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/heatMap/HeatMapOpDesc.scala index 426b0115c37..37445894a04 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/heatMap/HeatMapOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/heatMap/HeatMapOpDesc.scala @@ -3,7 +3,7 @@ package edu.uci.ics.texera.workflow.operators.visualization.heatMap import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/hierarchychart/HierarchyChartOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/hierarchychart/HierarchyChartOpDesc.scala index 4b179acdf4d..e4e020e5aac 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/hierarchychart/HierarchyChartOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/hierarchychart/HierarchyChartOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/histogram/HistogramChartOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/histogram/HistogramChartOpDesc.scala index f301c72f7af..70eba5e1561 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/histogram/HistogramChartOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/histogram/HistogramChartOpDesc.scala @@ -3,7 +3,7 @@ package edu.uci.ics.texera.workflow.operators.visualization.histogram import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/htmlviz/HtmlVizOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/htmlviz/HtmlVizOpDesc.scala index fdeb9587863..2c18e0e8a1d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/htmlviz/HtmlVizOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/htmlviz/HtmlVizOpDesc.scala @@ -5,10 +5,10 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/lineChart/LineChartOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/lineChart/LineChartOpDesc.scala index f4b9fc20c46..7886cf3ca8f 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/lineChart/LineChartOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/lineChart/LineChartOpDesc.scala @@ -5,7 +5,7 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/pieChart/PieChartOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/pieChart/PieChartOpDesc.scala index 55b89470702..ae46c7d1565 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/pieChart/PieChartOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/pieChart/PieChartOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/quiverPlot/QuiverPlotOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/quiverPlot/QuiverPlotOpDesc.scala index 860eb8ea6d7..7e06ed02de9 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/quiverPlot/QuiverPlotOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/quiverPlot/QuiverPlotOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/sankeyDiagram/SankeyDiagramOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/sankeyDiagram/SankeyDiagramOpDesc.scala index 8aa7b1e8ae0..e8ebc4b6d08 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/sankeyDiagram/SankeyDiagramOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/sankeyDiagram/SankeyDiagramOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/scatter3DChart/Scatter3dChartOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/scatter3DChart/Scatter3dChartOpDesc.scala index a7aab0bb3cc..704a3e4dae0 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/scatter3DChart/Scatter3dChartOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/scatter3DChart/Scatter3dChartOpDesc.scala @@ -3,7 +3,7 @@ package edu.uci.ics.texera.workflow.operators.visualization.scatter3DChart import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import com.kjetland.jackson.jsonSchema.annotations.{JsonSchemaInject, JsonSchemaTitle} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/scatterplot/ScatterplotOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/scatterplot/ScatterplotOpDesc.scala index 79b0215da2a..86bbee61634 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/scatterplot/ScatterplotOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/scatterplot/ScatterplotOpDesc.scala @@ -3,7 +3,7 @@ package edu.uci.ics.texera.workflow.operators.visualization.scatterplot import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import com.kjetland.jackson.jsonSchema.annotations.{JsonSchemaInject, JsonSchemaTitle} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/tablesChart/TablesPlotOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/tablesChart/TablesPlotOpDesc.scala index 1d97826e00f..9961ff5b0fc 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/tablesChart/TablesPlotOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/tablesChart/TablesPlotOpDesc.scala @@ -4,7 +4,7 @@ import com.fasterxml.jackson.annotation.{JsonProperty, JsonPropertyDescription} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ternaryPlot/TernaryPlotOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ternaryPlot/TernaryPlotOpDesc.scala index 34df82fd4d0..3ab93c7f26b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ternaryPlot/TernaryPlotOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/ternaryPlot/TernaryPlotOpDesc.scala @@ -6,7 +6,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Sc import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/urlviz/UrlVizOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/urlviz/UrlVizOpDesc.scala index ad748577b3b..89a250ecbc9 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/urlviz/UrlVizOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/urlviz/UrlVizOpDesc.scala @@ -5,10 +5,10 @@ import com.kjetland.jackson.jsonSchema.annotations.{JsonSchemaInject, JsonSchema import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, WorkflowIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/waterfallChart/WaterfallOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/waterfallChart/WaterfallOpDesc.scala index 13c7fc37707..c64e2a0bfad 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/waterfallChart/WaterfallOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/waterfallChart/WaterfallOpDesc.scala @@ -5,7 +5,7 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.operators.visualization.{ VisualizationConstants, VisualizationOperator diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/wordCloud/WordCloudOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/wordCloud/WordCloudOpDesc.scala index edf18d4d726..722bae1f123 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/wordCloud/WordCloudOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/visualization/wordCloud/WordCloudOpDesc.scala @@ -7,7 +7,7 @@ import com.kjetland.jackson.jsonSchema.annotations.{ JsonSchemaTitle } import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeName import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.PythonOperatorDescriptor diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/ControlPayloadV2.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/ControlPayloadV2.scala similarity index 60% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/ControlPayloadV2.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/ControlPayloadV2.scala index 0278dd23000..6bdc15c906b 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/ControlPayloadV2.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/ControlPayloadV2.scala @@ -3,11 +3,11 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.ambermessage +package edu.uci.ics.amber.engine.architecture.python @SerialVersionUID(0L) final case class ControlPayloadV2( - value: edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value + value: edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ControlPayloadV2] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -46,12 +46,12 @@ final case class ControlPayloadV2( __m.writeTo(_output__) }; } - def getControlInvocation: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation = value.controlInvocation.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation.defaultInstance) - def withControlInvocation(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation): ControlPayloadV2 = copy(value = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.ControlInvocation(__v)) - def getReturnInvocation: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation = value.returnInvocation.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation.defaultInstance) - def withReturnInvocation(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation): ControlPayloadV2 = copy(value = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.ReturnInvocation(__v)) - def clearValue: ControlPayloadV2 = copy(value = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.Empty) - def withValue(__v: edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value): ControlPayloadV2 = copy(value = __v) + def getControlInvocation: edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation = value.controlInvocation.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation.defaultInstance) + def withControlInvocation(__v: edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation): ControlPayloadV2 = copy(value = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.ControlInvocation(__v)) + def getReturnInvocation: edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation = value.returnInvocation.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation.defaultInstance) + def withReturnInvocation(__v: edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation): ControlPayloadV2 = copy(value = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.ReturnInvocation(__v)) + def clearValue: ControlPayloadV2 = copy(value = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.Empty) + def withValue(__v: edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value): ControlPayloadV2 = copy(value = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => value.controlInvocation.orNull @@ -66,66 +66,66 @@ final case class ControlPayloadV2( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.type = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2 - // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ControlPayloadV2]) + def companion: edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.type = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2 + // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2]) } -object ControlPayloadV2 extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2 = { - var __value: edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.Empty +object ControlPayloadV2 extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2 = { + var __value: edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.Empty var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __value = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.ControlInvocation(__value.controlInvocation.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __value = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.ControlInvocation(__value.controlInvocation.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => - __value = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.ReturnInvocation(__value.returnInvocation.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __value = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.ReturnInvocation(__value.returnInvocation.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2( + edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2( value = __value ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2( - value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation]]).map(edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.ControlInvocation(_)) - .orElse[edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value](__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation]]).map(edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.ReturnInvocation(_))) - .getOrElse(edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.Empty) + edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2( + value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation]]).map(edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.ControlInvocation(_)) + .orElse[edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value](__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation]]).map(edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.ReturnInvocation(_))) + .getOrElse(edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.Empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = AmbermessageProto.javaDescriptor.getMessageTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = AmbermessageProto.scalaDescriptor.messages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ProxyMessageProto.javaDescriptor.getMessageTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ProxyMessageProto.scalaDescriptor.messages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation - case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation + case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation + case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2( - value = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.Empty + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2( + value = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.Empty ) sealed trait Value extends _root_.scalapb.GeneratedOneof { def isEmpty: _root_.scala.Boolean = false def isDefined: _root_.scala.Boolean = true def isControlInvocation: _root_.scala.Boolean = false def isReturnInvocation: _root_.scala.Boolean = false - def controlInvocation: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] = _root_.scala.None - def returnInvocation: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation] = _root_.scala.None + def controlInvocation: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] = _root_.scala.None + def returnInvocation: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation] = _root_.scala.None } object Value { @SerialVersionUID(0L) - case object Empty extends edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value { + case object Empty extends edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value { type ValueType = _root_.scala.Nothing override def isEmpty: _root_.scala.Boolean = true override def isDefined: _root_.scala.Boolean = false @@ -134,31 +134,31 @@ object ControlPayloadV2 extends scalapb.GeneratedMessageCompanion[edu.uci.ics.am } @SerialVersionUID(0L) - final case class ControlInvocation(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation) extends edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation + final case class ControlInvocation(value: edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation) extends edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation override def isControlInvocation: _root_.scala.Boolean = true - override def controlInvocation: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] = Some(value) + override def controlInvocation: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] = Some(value) override def number: _root_.scala.Int = 1 } @SerialVersionUID(0L) - final case class ReturnInvocation(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation) extends edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation + final case class ReturnInvocation(value: edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation) extends edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation override def isReturnInvocation: _root_.scala.Boolean = true - override def returnInvocation: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation] = Some(value) + override def returnInvocation: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation] = Some(value) override def number: _root_.scala.Int = 2 } } - implicit class ControlPayloadV2Lens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2](_l) { - def controlInvocation: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] = field(_.getControlInvocation)((c_, f_) => c_.copy(value = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.ControlInvocation(f_))) - def returnInvocation: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation] = field(_.getReturnInvocation)((c_, f_) => c_.copy(value = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value.ReturnInvocation(f_))) - def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value] = field(_.value)((c_, f_) => c_.copy(value = f_)) + implicit class ControlPayloadV2Lens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2](_l) { + def controlInvocation: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] = field(_.getControlInvocation)((c_, f_) => c_.copy(value = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.ControlInvocation(f_))) + def returnInvocation: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation] = field(_.getReturnInvocation)((c_, f_) => c_.copy(value = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.ReturnInvocation(f_))) + def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value] = field(_.value)((c_, f_) => c_.copy(value = f_)) } final val CONTROL_INVOCATION_FIELD_NUMBER = 1 final val RETURN_INVOCATION_FIELD_NUMBER = 2 def of( - value: edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.Value - ): _root_.edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2 = _root_.edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2( + value: edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value + ): _root_.edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2 = _root_.edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2( value ) - // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ControlPayloadV2]) + // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2]) } diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/ProxyMessageProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/ProxyMessageProto.scala new file mode 100644 index 00000000000..8e8e320bd95 --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/ProxyMessageProto.scala @@ -0,0 +1,58 @@ +// Generated by the Scala Plugin for the Protocol Buffer Compiler. +// Do not edit! +// +// Protofile syntax: PROTO3 + +package edu.uci.ics.amber.engine.architecture.python + +object ProxyMessageProto extends _root_.scalapb.GeneratedFileObject { + lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( + edu.uci.ics.amber.engine.architecture.rpc.ControlCommandsProto, + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnsProto, + edu.uci.ics.amber.engine.common.VirtualIdentityProto, + edu.uci.ics.amber.engine.common.ActorMessageProto, + scalapb.options.ScalapbProto + ) + lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = + Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( + edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2, + edu.uci.ics.amber.engine.architecture.python.PythonDataHeader, + edu.uci.ics.amber.engine.architecture.python.PythonControlMessage, + edu.uci.ics.amber.engine.architecture.python.PythonActorMessage + ) + private lazy val ProtoBytes: _root_.scala.Array[Byte] = + scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( + """CkBlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3B5dGhvbi9wcm94eV9tZXNzYWdlLnByb3RvEixlZHUud + WNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnB5dGhvbhpAZWR1L3VjaS9pY3MvYW1iZXIvZW5naW5lL2FyY2hpdGVjd + HVyZS9ycGMvY29udHJvbF9jb21tYW5kcy5wcm90bxo/ZWR1L3VjaS9pY3MvYW1iZXIvZW5naW5lL2FyY2hpdGVjdHVyZS9ycGMvY + 29udHJvbF9yZXR1cm5zLnByb3RvGjZlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3ZpcnR1YWxfaWRlbnRpdHkucHJvd + G8aM2VkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9jb21tb24vYWN0b3JfbWVzc2FnZS5wcm90bxoVc2NhbGFwYi9zY2FsYXBiLnByb + 3RvIqcCChBDb250cm9sUGF5bG9hZFYyEoUBChJjb250cm9sX2ludm9jYXRpb24YASABKAsyPC5lZHUudWNpLmljcy5hbWJlci5lb + mdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Db250cm9sSW52b2NhdGlvbkIW4j8TEhFjb250cm9sSW52b2NhdGlvbkgAUhFjb250cm9sS + W52b2NhdGlvbhKBAQoRcmV0dXJuX2ludm9jYXRpb24YAiABKAsyOy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0d + XJlLnJwYy5SZXR1cm5JbnZvY2F0aW9uQhXiPxISEHJldHVybkludm9jYXRpb25IAFIQcmV0dXJuSW52b2NhdGlvbkIHCgV2YWx1Z + SKdAQoQUHl0aG9uRGF0YUhlYWRlchJUCgN0YWcYASABKAsyNS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkFjdG9yV + mlydHVhbElkZW50aXR5QgviPwgSA3RhZ/ABAVIDdGFnEjMKDHBheWxvYWRfdHlwZRgCIAEoCUIQ4j8NEgtwYXlsb2FkVHlwZVILc + GF5bG9hZFR5cGUi1wEKFFB5dGhvbkNvbnRyb2xNZXNzYWdlElQKA3RhZxgBIAEoCzI1LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZ + S5jb21tb24uQWN0b3JWaXJ0dWFsSWRlbnRpdHlCC+I/CBIDdGFn8AEBUgN0YWcSaQoHcGF5bG9hZBgCIAEoCzI+LmVkdS51Y2kua + WNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucHl0aG9uLkNvbnRyb2xQYXlsb2FkVjJCD+I/DBIHcGF5bG9hZPABAVIHcGF5b + G9hZCJuChJQeXRob25BY3Rvck1lc3NhZ2USWAoHcGF5bG9hZBgBIAEoCzItLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb + 24uQWN0b3JDb21tYW5kQg/iPwwSB3BheWxvYWTwAQFSB3BheWxvYWRCC+I/CBABSABYAHgBYgZwcm90bzM=""" + ).mkString) + lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { + val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) + _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) + } + lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { + val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) + com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( + edu.uci.ics.amber.engine.architecture.rpc.ControlCommandsProto.javaDescriptor, + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnsProto.javaDescriptor, + edu.uci.ics.amber.engine.common.VirtualIdentityProto.javaDescriptor, + edu.uci.ics.amber.engine.common.ActorMessageProto.javaDescriptor, + scalapb.options.ScalapbProto.javaDescriptor + )) + } + @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") + def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/actormessage/PythonActorMessage.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/PythonActorMessage.scala similarity index 56% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/actormessage/PythonActorMessage.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/PythonActorMessage.scala index 9d4b717d432..caee0f3d2cc 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/actormessage/PythonActorMessage.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/PythonActorMessage.scala @@ -3,11 +3,11 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.actormessage +package edu.uci.ics.amber.engine.architecture.python @SerialVersionUID(0L) final case class PythonActorMessage( - payload: edu.uci.ics.amber.engine.common.actormessage.ActorCommand + payload: edu.uci.ics.amber.engine.common.ActorCommand ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[PythonActorMessage] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -15,7 +15,7 @@ final case class PythonActorMessage( var __size = 0 { - val __value = edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage._typemapper_payload.toBase(payload) + val __value = edu.uci.ics.amber.engine.architecture.python.PythonActorMessage._typemapper_payload.toBase(payload) if (__value.serializedSize != 0) { __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } @@ -33,7 +33,7 @@ final case class PythonActorMessage( } def writeTo(`_output__`: _root_.com.google.protobuf.CodedOutputStream): _root_.scala.Unit = { { - val __v = edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage._typemapper_payload.toBase(payload) + val __v = edu.uci.ics.amber.engine.architecture.python.PythonActorMessage._typemapper_payload.toBase(payload) if (__v.serializedSize != 0) { _output__.writeTag(1, 2) _output__.writeUInt32NoTag(__v.serializedSize) @@ -41,76 +41,76 @@ final case class PythonActorMessage( } }; } - def withPayload(__v: edu.uci.ics.amber.engine.common.actormessage.ActorCommand): PythonActorMessage = copy(payload = __v) + def withPayload(__v: edu.uci.ics.amber.engine.common.ActorCommand): PythonActorMessage = copy(payload = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { - val __t = edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage._typemapper_payload.toBase(payload) - if (__t != edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.defaultInstance) __t else null + val __t = edu.uci.ics.amber.engine.architecture.python.PythonActorMessage._typemapper_payload.toBase(payload) + if (__t != edu.uci.ics.amber.engine.common.ActorCommandMessage.defaultInstance) __t else null } } } def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = { _root_.scala.Predef.require(__field.containingMessage eq companion.scalaDescriptor) (__field.number: @_root_.scala.unchecked) match { - case 1 => edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage._typemapper_payload.toBase(payload).toPMessage + case 1 => edu.uci.ics.amber.engine.architecture.python.PythonActorMessage._typemapper_payload.toBase(payload).toPMessage } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage.type = edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage - // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.PythonActorMessage]) + def companion: edu.uci.ics.amber.engine.architecture.python.PythonActorMessage.type = edu.uci.ics.amber.engine.architecture.python.PythonActorMessage + // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.python.PythonActorMessage]) } -object PythonActorMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage = { - var __payload: _root_.scala.Option[edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage] = _root_.scala.None +object PythonActorMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.python.PythonActorMessage] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.python.PythonActorMessage] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.python.PythonActorMessage = { + var __payload: _root_.scala.Option[edu.uci.ics.amber.engine.common.ActorCommandMessage] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __payload = _root_.scala.Some(__payload.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __payload = _root_.scala.Some(__payload.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorCommandMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage( - payload = edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage._typemapper_payload.toCustom(__payload.getOrElse(edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.defaultInstance)) + edu.uci.ics.amber.engine.architecture.python.PythonActorMessage( + payload = edu.uci.ics.amber.engine.architecture.python.PythonActorMessage._typemapper_payload.toCustom(__payload.getOrElse(edu.uci.ics.amber.engine.common.ActorCommandMessage.defaultInstance)) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.python.PythonActorMessage] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage( - payload = edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage._typemapper_payload.toCustom(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage]).getOrElse(edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.defaultInstance)) + edu.uci.ics.amber.engine.architecture.python.PythonActorMessage( + payload = edu.uci.ics.amber.engine.architecture.python.PythonActorMessage._typemapper_payload.toCustom(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.ActorCommandMessage]).getOrElse(edu.uci.ics.amber.engine.common.ActorCommandMessage.defaultInstance)) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ActormessageProto.javaDescriptor.getMessageTypes().get(3) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ActormessageProto.scalaDescriptor.messages(3) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ProxyMessageProto.javaDescriptor.getMessageTypes().get(3) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ProxyMessageProto.scalaDescriptor.messages(3) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage + case 1 => __out = edu.uci.ics.amber.engine.common.ActorCommandMessage } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage( - payload = edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage._typemapper_payload.toCustom(edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.defaultInstance) + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.python.PythonActorMessage( + payload = edu.uci.ics.amber.engine.architecture.python.PythonActorMessage._typemapper_payload.toCustom(edu.uci.ics.amber.engine.common.ActorCommandMessage.defaultInstance) ) - implicit class PythonActorMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage](_l) { - def payload: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.actormessage.ActorCommand] = field(_.payload)((c_, f_) => c_.copy(payload = f_)) + implicit class PythonActorMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.python.PythonActorMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.python.PythonActorMessage](_l) { + def payload: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ActorCommand] = field(_.payload)((c_, f_) => c_.copy(payload = f_)) } final val PAYLOAD_FIELD_NUMBER = 1 @transient - private[actormessage] val _typemapper_payload: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage, edu.uci.ics.amber.engine.common.actormessage.ActorCommand] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage, edu.uci.ics.amber.engine.common.actormessage.ActorCommand]] + private[python] val _typemapper_payload: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ActorCommandMessage, edu.uci.ics.amber.engine.common.ActorCommand] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ActorCommandMessage, edu.uci.ics.amber.engine.common.ActorCommand]] def of( - payload: edu.uci.ics.amber.engine.common.actormessage.ActorCommand - ): _root_.edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage = _root_.edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage( + payload: edu.uci.ics.amber.engine.common.ActorCommand + ): _root_.edu.uci.ics.amber.engine.architecture.python.PythonActorMessage = _root_.edu.uci.ics.amber.engine.architecture.python.PythonActorMessage( payload ) - // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.PythonActorMessage]) + // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.python.PythonActorMessage]) } diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/PythonControlMessage.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/PythonControlMessage.scala similarity index 58% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/PythonControlMessage.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/PythonControlMessage.scala index 00fb34a6022..a2d19f3826d 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/PythonControlMessage.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/PythonControlMessage.scala @@ -3,12 +3,12 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.ambermessage +package edu.uci.ics.amber.engine.architecture.python @SerialVersionUID(0L) final case class PythonControlMessage( - tag: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity, - payload: edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2 + tag: edu.uci.ics.amber.engine.common.ActorVirtualIdentity, + payload: edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2 ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[PythonControlMessage] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -57,17 +57,17 @@ final case class PythonControlMessage( } }; } - def withTag(__v: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity): PythonControlMessage = copy(tag = __v) - def withPayload(__v: edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2): PythonControlMessage = copy(payload = __v) + def withTag(__v: edu.uci.ics.amber.engine.common.ActorVirtualIdentity): PythonControlMessage = copy(tag = __v) + def withPayload(__v: edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2): PythonControlMessage = copy(payload = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = tag - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) __t else null } case 2 => { val __t = payload - if (__t != edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.defaultInstance) __t else null } } } @@ -79,69 +79,69 @@ final case class PythonControlMessage( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage.type = edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage - // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.PythonControlMessage]) + def companion: edu.uci.ics.amber.engine.architecture.python.PythonControlMessage.type = edu.uci.ics.amber.engine.architecture.python.PythonControlMessage + // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.python.PythonControlMessage]) } -object PythonControlMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage = { - var __tag: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = _root_.scala.None - var __payload: _root_.scala.Option[edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2] = _root_.scala.None +object PythonControlMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.python.PythonControlMessage] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.python.PythonControlMessage] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.python.PythonControlMessage = { + var __tag: _root_.scala.Option[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = _root_.scala.None + var __payload: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __tag = _root_.scala.Some(__tag.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __tag = _root_.scala.Some(__tag.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => - __payload = _root_.scala.Some(__payload.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __payload = _root_.scala.Some(__payload.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage( - tag = __tag.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance), - payload = __payload.getOrElse(edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.defaultInstance) + edu.uci.ics.amber.engine.architecture.python.PythonControlMessage( + tag = __tag.getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance), + payload = __payload.getOrElse(edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.defaultInstance) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.python.PythonControlMessage] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage( - tag = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance), - payload = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2]).getOrElse(edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.defaultInstance) + edu.uci.ics.amber.engine.architecture.python.PythonControlMessage( + tag = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance), + payload = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2]).getOrElse(edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.defaultInstance) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = AmbermessageProto.javaDescriptor.getMessageTypes().get(2) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = AmbermessageProto.scalaDescriptor.messages(2) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ProxyMessageProto.javaDescriptor.getMessageTypes().get(2) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ProxyMessageProto.scalaDescriptor.messages(2) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity - case 2 => __out = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2 + case 1 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity + case 2 => __out = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2 } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage( - tag = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance, - payload = edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2.defaultInstance + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.python.PythonControlMessage( + tag = edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance, + payload = edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.defaultInstance ) - implicit class PythonControlMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage](_l) { - def tag: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = field(_.tag)((c_, f_) => c_.copy(tag = f_)) - def payload: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2] = field(_.payload)((c_, f_) => c_.copy(payload = f_)) + implicit class PythonControlMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.python.PythonControlMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.python.PythonControlMessage](_l) { + def tag: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = field(_.tag)((c_, f_) => c_.copy(tag = f_)) + def payload: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2] = field(_.payload)((c_, f_) => c_.copy(payload = f_)) } final val TAG_FIELD_NUMBER = 1 final val PAYLOAD_FIELD_NUMBER = 2 def of( - tag: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity, - payload: edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2 - ): _root_.edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage = _root_.edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage( + tag: edu.uci.ics.amber.engine.common.ActorVirtualIdentity, + payload: edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2 + ): _root_.edu.uci.ics.amber.engine.architecture.python.PythonControlMessage = _root_.edu.uci.ics.amber.engine.architecture.python.PythonControlMessage( tag, payload ) - // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.PythonControlMessage]) + // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.python.PythonControlMessage]) } diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/PythonDataHeader.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/PythonDataHeader.scala similarity index 68% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/PythonDataHeader.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/PythonDataHeader.scala index 67ffb1d3b9a..6b5c8fc1cea 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/PythonDataHeader.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/python/PythonDataHeader.scala @@ -3,11 +3,11 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.ambermessage +package edu.uci.ics.amber.engine.architecture.python @SerialVersionUID(0L) final case class PythonDataHeader( - tag: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity, + tag: edu.uci.ics.amber.engine.common.ActorVirtualIdentity, payloadType: _root_.scala.Predef.String ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[PythonDataHeader] { @transient @@ -55,13 +55,13 @@ final case class PythonDataHeader( } }; } - def withTag(__v: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity): PythonDataHeader = copy(tag = __v) + def withTag(__v: edu.uci.ics.amber.engine.common.ActorVirtualIdentity): PythonDataHeader = copy(tag = __v) def withPayloadType(__v: _root_.scala.Predef.String): PythonDataHeader = copy(payloadType = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = tag - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) __t else null } case 2 => { val __t = payloadType @@ -77,14 +77,14 @@ final case class PythonDataHeader( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader.type = edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader - // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.PythonDataHeader]) + def companion: edu.uci.ics.amber.engine.architecture.python.PythonDataHeader.type = edu.uci.ics.amber.engine.architecture.python.PythonDataHeader + // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.python.PythonDataHeader]) } -object PythonDataHeader extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader = { - var __tag: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = _root_.scala.None +object PythonDataHeader extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.python.PythonDataHeader] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.python.PythonDataHeader] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.python.PythonDataHeader = { + var __tag: _root_.scala.Option[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = _root_.scala.None var __payloadType: _root_.scala.Predef.String = "" var _done__ = false while (!_done__) { @@ -92,53 +92,53 @@ object PythonDataHeader extends scalapb.GeneratedMessageCompanion[edu.uci.ics.am _tag__ match { case 0 => _done__ = true case 10 => - __tag = _root_.scala.Some(__tag.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __tag = _root_.scala.Some(__tag.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => __payloadType = _input__.readStringRequireUtf8() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader( - tag = __tag.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.python.PythonDataHeader( + tag = __tag.getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance), payloadType = __payloadType ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.python.PythonDataHeader] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader( - tag = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.python.PythonDataHeader( + tag = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance), payloadType = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = AmbermessageProto.javaDescriptor.getMessageTypes().get(1) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = AmbermessageProto.scalaDescriptor.messages(1) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ProxyMessageProto.javaDescriptor.getMessageTypes().get(1) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ProxyMessageProto.scalaDescriptor.messages(1) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader( - tag = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance, + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.python.PythonDataHeader( + tag = edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance, payloadType = "" ) - implicit class PythonDataHeaderLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader](_l) { - def tag: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = field(_.tag)((c_, f_) => c_.copy(tag = f_)) + implicit class PythonDataHeaderLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.python.PythonDataHeader]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.python.PythonDataHeader](_l) { + def tag: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = field(_.tag)((c_, f_) => c_.copy(tag = f_)) def payloadType: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.payloadType)((c_, f_) => c_.copy(payloadType = f_)) } final val TAG_FIELD_NUMBER = 1 final val PAYLOAD_TYPE_FIELD_NUMBER = 2 def of( - tag: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity, + tag: edu.uci.ics.amber.engine.common.ActorVirtualIdentity, payloadType: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader = _root_.edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader( + ): _root_.edu.uci.ics.amber.engine.architecture.python.PythonDataHeader = _root_.edu.uci.ics.amber.engine.architecture.python.PythonDataHeader( tag, payloadType ) - // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.PythonDataHeader]) + // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.python.PythonDataHeader]) } diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/AsyncRPCContext.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/AsyncRPCContext.scala similarity index 58% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/AsyncRPCContext.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/AsyncRPCContext.scala index a182d51d9a5..589ec85dc69 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/AsyncRPCContext.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/AsyncRPCContext.scala @@ -3,12 +3,12 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlcommands +package edu.uci.ics.amber.engine.architecture.rpc @SerialVersionUID(0L) final case class AsyncRPCContext( - sender: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity, - receiver: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity + sender: edu.uci.ics.amber.engine.common.ActorVirtualIdentity, + receiver: edu.uci.ics.amber.engine.common.ActorVirtualIdentity ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[AsyncRPCContext] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -57,17 +57,17 @@ final case class AsyncRPCContext( } }; } - def withSender(__v: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity): AsyncRPCContext = copy(sender = __v) - def withReceiver(__v: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity): AsyncRPCContext = copy(receiver = __v) + def withSender(__v: edu.uci.ics.amber.engine.common.ActorVirtualIdentity): AsyncRPCContext = copy(sender = __v) + def withReceiver(__v: edu.uci.ics.amber.engine.common.ActorVirtualIdentity): AsyncRPCContext = copy(receiver = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = sender - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) __t else null } case 2 => { val __t = receiver - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) __t else null } } } @@ -79,67 +79,67 @@ final case class AsyncRPCContext( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext + def companion: edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext.type = edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext]) } -object AsyncRPCContext extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext = { - var __sender: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = _root_.scala.None - var __receiver: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = _root_.scala.None +object AsyncRPCContext extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext = { + var __sender: _root_.scala.Option[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = _root_.scala.None + var __receiver: _root_.scala.Option[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __sender = _root_.scala.Some(__sender.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sender = _root_.scala.Some(__sender.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => - __receiver = _root_.scala.Some(__receiver.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __receiver = _root_.scala.Some(__receiver.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext( - sender = __sender.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance), - receiver = __receiver.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) + edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext( + sender = __sender.getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance), + receiver = __receiver.getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext( - sender = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance), - receiver = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) + edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext( + sender = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance), + receiver = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(2) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(2) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(2) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(2) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity - case 2 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity + case 2 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext( - sender = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance, - receiver = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext( + sender = edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance, + receiver = edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance ) - implicit class AsyncRPCContextLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext](_l) { - def sender: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = field(_.sender)((c_, f_) => c_.copy(sender = f_)) - def receiver: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = field(_.receiver)((c_, f_) => c_.copy(receiver = f_)) + implicit class AsyncRPCContextLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext](_l) { + def sender: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = field(_.sender)((c_, f_) => c_.copy(sender = f_)) + def receiver: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = field(_.receiver)((c_, f_) => c_.copy(receiver = f_)) } final val SENDER_FIELD_NUMBER = 1 final val RECEIVER_FIELD_NUMBER = 2 def of( - sender: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity, - receiver: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext( + sender: edu.uci.ics.amber.engine.common.ActorVirtualIdentity, + receiver: edu.uci.ics.amber.engine.common.ActorVirtualIdentity + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext = _root_.edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext( sender, receiver ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ChannelMarkerPayload.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ChannelMarkerPayload.scala similarity index 57% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ChannelMarkerPayload.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ChannelMarkerPayload.scala index d6041dd9386..5361a7f55d0 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ChannelMarkerPayload.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ChannelMarkerPayload.scala @@ -3,16 +3,16 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlcommands +package edu.uci.ics.amber.engine.architecture.rpc /** Message for ChannelMarkerPayload */ @SerialVersionUID(0L) final case class ChannelMarkerPayload( - id: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity, - markerType: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType, - scope: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity], - commandMapping: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] + id: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity, + markerType: edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType, + scope: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity], + commandMapping: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ChannelMarkerPayload] with edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessagePayload { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -37,7 +37,7 @@ final case class ChannelMarkerPayload( __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } commandMapping.foreach { __item => - val __value = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload._typemapper_commandMapping.toBase(__item) + val __value = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload._typemapper_commandMapping.toBase(__item) __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } __size @@ -73,34 +73,34 @@ final case class ChannelMarkerPayload( __m.writeTo(_output__) }; commandMapping.foreach { __v => - val __m = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload._typemapper_commandMapping.toBase(__v) + val __m = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload._typemapper_commandMapping.toBase(__v) _output__.writeTag(4, 2) _output__.writeUInt32NoTag(__m.serializedSize) __m.writeTo(_output__) }; } - def withId(__v: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity): ChannelMarkerPayload = copy(id = __v) - def withMarkerType(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType): ChannelMarkerPayload = copy(markerType = __v) + def withId(__v: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity): ChannelMarkerPayload = copy(id = __v) + def withMarkerType(__v: edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType): ChannelMarkerPayload = copy(markerType = __v) def clearScope = copy(scope = _root_.scala.Seq.empty) - def addScope(__vs: edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity *): ChannelMarkerPayload = addAllScope(__vs) - def addAllScope(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]): ChannelMarkerPayload = copy(scope = scope ++ __vs) - def withScope(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]): ChannelMarkerPayload = copy(scope = __v) + def addScope(__vs: edu.uci.ics.amber.engine.common.ChannelIdentity *): ChannelMarkerPayload = addAllScope(__vs) + def addAllScope(__vs: Iterable[edu.uci.ics.amber.engine.common.ChannelIdentity]): ChannelMarkerPayload = copy(scope = scope ++ __vs) + def withScope(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]): ChannelMarkerPayload = copy(scope = __v) def clearCommandMapping = copy(commandMapping = _root_.scala.collection.immutable.Map.empty) - def addCommandMapping(__vs: (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation) *): ChannelMarkerPayload = addAllCommandMapping(__vs) - def addAllCommandMapping(__vs: Iterable[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation)]): ChannelMarkerPayload = copy(commandMapping = commandMapping ++ __vs) - def withCommandMapping(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation]): ChannelMarkerPayload = copy(commandMapping = __v) + def addCommandMapping(__vs: (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation) *): ChannelMarkerPayload = addAllCommandMapping(__vs) + def addAllCommandMapping(__vs: Iterable[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation)]): ChannelMarkerPayload = copy(commandMapping = commandMapping ++ __vs) + def withCommandMapping(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation]): ChannelMarkerPayload = copy(commandMapping = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = id - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance) __t else null } case 2 => { val __t = markerType.javaValueDescriptor if (__t.getNumber() != 0) __t else null } case 3 => scope - case 4 => commandMapping.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload._typemapper_commandMapping.toBase(_)).toSeq + case 4 => commandMapping.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload._typemapper_commandMapping.toBase(_)).toSeq } } def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = { @@ -109,85 +109,85 @@ final case class ChannelMarkerPayload( case 1 => id.toPMessage case 2 => _root_.scalapb.descriptors.PEnum(markerType.scalaValueDescriptor) case 3 => _root_.scalapb.descriptors.PRepeated(scope.iterator.map(_.toPMessage).toVector) - case 4 => _root_.scalapb.descriptors.PRepeated(commandMapping.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload._typemapper_commandMapping.toBase(_).toPMessage).toVector) + case 4 => _root_.scalapb.descriptors.PRepeated(commandMapping.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload._typemapper_commandMapping.toBase(_).toPMessage).toVector) } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload + def companion: edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.type = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload]) } -object ChannelMarkerPayload extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload = { - var __id: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] = _root_.scala.None - var __markerType: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.REQUIRE_ALIGNMENT - val __scope: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] - val __commandMapping: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] +object ChannelMarkerPayload extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload = { + var __id: _root_.scala.Option[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] = _root_.scala.None + var __markerType: edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.REQUIRE_ALIGNMENT + val __scope: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ChannelIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ChannelIdentity] + val __commandMapping: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __id = _root_.scala.Some(__id.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __id = _root_.scala.Some(__id.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 16 => - __markerType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.fromValue(_input__.readEnum()) + __markerType = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.fromValue(_input__.readEnum()) case 26 => - __scope += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity](_input__) + __scope += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ChannelIdentity](_input__) case 34 => - __commandMapping += edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload._typemapper_commandMapping.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry](_input__)) + __commandMapping += edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload._typemapper_commandMapping.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry](_input__)) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload( - id = __id.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload( + id = __id.getOrElse(edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance), markerType = __markerType, scope = __scope.result(), commandMapping = __commandMapping.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload( - id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance), - markerType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.REQUIRE_ALIGNMENT.scalaValueDescriptor).number), - scope = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]]).getOrElse(_root_.scala.Seq.empty), - commandMapping = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload._typemapper_commandMapping.toCustom(_)).toMap + edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload( + id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance), + markerType = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.REQUIRE_ALIGNMENT.scalaValueDescriptor).number), + scope = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]]).getOrElse(_root_.scala.Seq.empty), + commandMapping = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload._typemapper_commandMapping.toCustom(_)).toMap ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(4) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(4) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(4) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(4) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity - case 3 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity - case 4 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry + case 1 => __out = edu.uci.ics.amber.engine.common.ChannelMarkerIdentity + case 3 => __out = edu.uci.ics.amber.engine.common.ChannelIdentity + case 4 => __out = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry + _root_.edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry ) def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = { (__fieldNumber: @_root_.scala.unchecked) match { - case 2 => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType + case 2 => edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType } } - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload( - id = edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance, - markerType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.REQUIRE_ALIGNMENT, + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload( + id = edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance, + markerType = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.REQUIRE_ALIGNMENT, scope = _root_.scala.Seq.empty, commandMapping = _root_.scala.collection.immutable.Map.empty ) @SerialVersionUID(0L) final case class CommandMappingEntry( key: _root_.scala.Predef.String, - value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] + value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[CommandMappingEntry] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -230,9 +230,9 @@ object ChannelMarkerPayload extends scalapb.GeneratedMessageCompanion[edu.uci.ic }; } def withKey(__v: _root_.scala.Predef.String): CommandMappingEntry = copy(key = __v) - def getValue: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation = value.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation.defaultInstance) + def getValue: edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation = value.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation.defaultInstance) def clearValue: CommandMappingEntry = copy(value = _root_.scala.None) - def withValue(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation): CommandMappingEntry = copy(value = Option(__v)) + def withValue(__v: edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation): CommandMappingEntry = copy(value = Option(__v)) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -250,15 +250,15 @@ object ChannelMarkerPayload extends scalapb.GeneratedMessageCompanion[edu.uci.ic } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry + def companion: edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry.type = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry]) } - object CommandMappingEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry = { + object CommandMappingEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry = { var __key: _root_.scala.Predef.String = "" - var __value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] = _root_.scala.None + var __value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -267,77 +267,77 @@ object ChannelMarkerPayload extends scalapb.GeneratedMessageCompanion[edu.uci.ic case 10 => __key = _input__.readStringRequireUtf8() case 18 => - __value = Option(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __value = Option(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry( + edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry( key = __key, value = __value ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry( + edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry( key = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), - value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation]]) + value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation]]) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.javaDescriptor.getNestedTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.scalaDescriptor.nestedMessages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.javaDescriptor.getNestedTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.scalaDescriptor.nestedMessages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation + case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry( key = "", value = _root_.scala.None ) - implicit class CommandMappingEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry](_l) { + implicit class CommandMappingEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry](_l) { def key: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.key)((c_, f_) => c_.copy(key = f_)) - def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] = field(_.getValue)((c_, f_) => c_.copy(value = Option(f_))) - def optionalValue: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation]] = field(_.value)((c_, f_) => c_.copy(value = f_)) + def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] = field(_.getValue)((c_, f_) => c_.copy(value = Option(f_))) + def optionalValue: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation]] = field(_.value)((c_, f_) => c_.copy(value = f_)) } final val KEY_FIELD_NUMBER = 1 final val VALUE_FIELD_NUMBER = 2 @transient - implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation)] = - _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation)](__m => (__m.key, __m.getValue))(__p => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry(__p._1, Some(__p._2))) + implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation)] = + _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation)](__m => (__m.key, __m.getValue))(__p => edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry(__p._1, Some(__p._2))) def of( key: _root_.scala.Predef.String, - value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry( + value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry = _root_.edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry( key, value ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry]) } - implicit class ChannelMarkerPayloadLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload](_l) { - def id: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] = field(_.id)((c_, f_) => c_.copy(id = f_)) - def markerType: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType] = field(_.markerType)((c_, f_) => c_.copy(markerType = f_)) - def scope: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]] = field(_.scope)((c_, f_) => c_.copy(scope = f_)) - def commandMapping: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation]] = field(_.commandMapping)((c_, f_) => c_.copy(commandMapping = f_)) + implicit class ChannelMarkerPayloadLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload](_l) { + def id: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] = field(_.id)((c_, f_) => c_.copy(id = f_)) + def markerType: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType] = field(_.markerType)((c_, f_) => c_.copy(markerType = f_)) + def scope: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]] = field(_.scope)((c_, f_) => c_.copy(scope = f_)) + def commandMapping: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation]] = field(_.commandMapping)((c_, f_) => c_.copy(commandMapping = f_)) } final val ID_FIELD_NUMBER = 1 - final val MARKERTYPE_FIELD_NUMBER = 2 + final val MARKER_TYPE_FIELD_NUMBER = 2 final val SCOPE_FIELD_NUMBER = 3 - final val COMMANDMAPPING_FIELD_NUMBER = 4 + final val COMMAND_MAPPING_FIELD_NUMBER = 4 @transient - private[controlcommands] val _typemapper_commandMapping: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload.CommandMappingEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation)]] + private[rpc] val _typemapper_commandMapping: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload.CommandMappingEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation)]] def of( - id: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity, - markerType: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType, - scope: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity], - commandMapping: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload( + id: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity, + markerType: edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType, + scope: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity], + commandMapping: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload = _root_.edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload( id, markerType, scope, diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ChannelMarkerType.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ChannelMarkerType.scala similarity index 78% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ChannelMarkerType.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ChannelMarkerType.scala index a2302811d05..946adf57cda 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ChannelMarkerType.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ChannelMarkerType.scala @@ -3,7 +3,7 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlcommands +package edu.uci.ics.amber.engine.architecture.rpc /** Enum for ChannelMarkerType */ @@ -11,8 +11,8 @@ sealed abstract class ChannelMarkerType(val value: _root_.scala.Int) extends _ro type EnumType = ChannelMarkerType def isRequireAlignment: _root_.scala.Boolean = false def isNoAlignment: _root_.scala.Boolean = false - def companion: _root_.scalapb.GeneratedEnumCompanion[ChannelMarkerType] = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType - final def asRecognized: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.Recognized] = if (isUnrecognized) _root_.scala.None else _root_.scala.Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.Recognized]) + def companion: _root_.scalapb.GeneratedEnumCompanion[ChannelMarkerType] = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType + final def asRecognized: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.Recognized] = if (isUnrecognized) _root_.scala.None else _root_.scala.Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.Recognized]) } object ChannelMarkerType extends _root_.scalapb.GeneratedEnumCompanion[ChannelMarkerType] { @@ -41,6 +41,6 @@ object ChannelMarkerType extends _root_.scalapb.GeneratedEnumCompanion[ChannelMa case 1 => NO_ALIGNMENT case __other => Unrecognized(__other) } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.EnumDescriptor = ControlcommandsProto.javaDescriptor.getEnumTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.EnumDescriptor = ControlcommandsProto.scalaDescriptor.enums(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.EnumDescriptor = ControlCommandsProto.javaDescriptor.getEnumTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.EnumDescriptor = ControlCommandsProto.scalaDescriptor.enums(0) } \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ConsoleMessage.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ConsoleMessage.scala similarity index 80% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ConsoleMessage.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ConsoleMessage.scala index fd87f7c2d22..76c855bbae2 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ConsoleMessage.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ConsoleMessage.scala @@ -3,13 +3,13 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlcommands +package edu.uci.ics.amber.engine.architecture.rpc @SerialVersionUID(0L) final case class ConsoleMessage( workerId: _root_.scala.Predef.String, timestamp: com.google.protobuf.timestamp.Timestamp, - msgType: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType, + msgType: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType, source: _root_.scala.Predef.String, title: _root_.scala.Predef.String, message: _root_.scala.Predef.String @@ -113,7 +113,7 @@ final case class ConsoleMessage( } def withWorkerId(__v: _root_.scala.Predef.String): ConsoleMessage = copy(workerId = __v) def withTimestamp(__v: com.google.protobuf.timestamp.Timestamp): ConsoleMessage = copy(timestamp = __v) - def withMsgType(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType): ConsoleMessage = copy(msgType = __v) + def withMsgType(__v: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType): ConsoleMessage = copy(msgType = __v) def withSource(__v: _root_.scala.Predef.String): ConsoleMessage = copy(source = __v) def withTitle(__v: _root_.scala.Predef.String): ConsoleMessage = copy(title = __v) def withMessage(__v: _root_.scala.Predef.String): ConsoleMessage = copy(message = __v) @@ -157,16 +157,16 @@ final case class ConsoleMessage( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage + def companion: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage.type = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage]) } -object ConsoleMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage = { +object ConsoleMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage = { var __workerId: _root_.scala.Predef.String = "" var __timestamp: _root_.scala.Option[com.google.protobuf.timestamp.Timestamp] = _root_.scala.None - var __msgType: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType.PRINT + var __msgType: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType.PRINT var __source: _root_.scala.Predef.String = "" var __title: _root_.scala.Predef.String = "" var __message: _root_.scala.Predef.String = "" @@ -180,7 +180,7 @@ object ConsoleMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.ambe case 18 => __timestamp = _root_.scala.Some(__timestamp.fold(_root_.scalapb.LiteParser.readMessage[com.google.protobuf.timestamp.Timestamp](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 24 => - __msgType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType.fromValue(_input__.readEnum()) + __msgType = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType.fromValue(_input__.readEnum()) case 34 => __source = _input__.readStringRequireUtf8() case 42 => @@ -190,7 +190,7 @@ object ConsoleMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.ambe case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage( + edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage( workerId = __workerId, timestamp = __timestamp.getOrElse(com.google.protobuf.timestamp.Timestamp.defaultInstance), msgType = __msgType, @@ -199,21 +199,21 @@ object ConsoleMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.ambe message = __message ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage( + edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage( workerId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), timestamp = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[com.google.protobuf.timestamp.Timestamp]).getOrElse(com.google.protobuf.timestamp.Timestamp.defaultInstance), - msgType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType.PRINT.scalaValueDescriptor).number), + msgType = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType.PRINT.scalaValueDescriptor).number), source = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), title = __fieldsMap.get(scalaDescriptor.findFieldByNumber(5).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), message = __fieldsMap.get(scalaDescriptor.findFieldByNumber(6).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(12) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(12) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(12) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(12) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { @@ -224,21 +224,21 @@ object ConsoleMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.ambe lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = { (__fieldNumber: @_root_.scala.unchecked) match { - case 3 => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType + case 3 => edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType } } - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage( workerId = "", timestamp = com.google.protobuf.timestamp.Timestamp.defaultInstance, - msgType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType.PRINT, + msgType = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType.PRINT, source = "", title = "", message = "" ) - implicit class ConsoleMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage](_l) { + implicit class ConsoleMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage](_l) { def workerId: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.workerId)((c_, f_) => c_.copy(workerId = f_)) def timestamp: _root_.scalapb.lenses.Lens[UpperPB, com.google.protobuf.timestamp.Timestamp] = field(_.timestamp)((c_, f_) => c_.copy(timestamp = f_)) - def msgType: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType] = field(_.msgType)((c_, f_) => c_.copy(msgType = f_)) + def msgType: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType] = field(_.msgType)((c_, f_) => c_.copy(msgType = f_)) def source: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.source)((c_, f_) => c_.copy(source = f_)) def title: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.title)((c_, f_) => c_.copy(title = f_)) def message: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.message)((c_, f_) => c_.copy(message = f_)) @@ -252,11 +252,11 @@ object ConsoleMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.ambe def of( workerId: _root_.scala.Predef.String, timestamp: com.google.protobuf.timestamp.Timestamp, - msgType: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType, + msgType: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType, source: _root_.scala.Predef.String, title: _root_.scala.Predef.String, message: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage = _root_.edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage( workerId, timestamp, msgType, diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ConsoleMessageType.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ConsoleMessageType.scala similarity index 81% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ConsoleMessageType.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ConsoleMessageType.scala index c67a98aba3e..c4b09da5639 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ConsoleMessageType.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ConsoleMessageType.scala @@ -3,7 +3,7 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlcommands +package edu.uci.ics.amber.engine.architecture.rpc sealed abstract class ConsoleMessageType(val value: _root_.scala.Int) extends _root_.scalapb.GeneratedEnum { type EnumType = ConsoleMessageType @@ -11,8 +11,8 @@ sealed abstract class ConsoleMessageType(val value: _root_.scala.Int) extends _r def isError: _root_.scala.Boolean = false def isCommand: _root_.scala.Boolean = false def isDebugger: _root_.scala.Boolean = false - def companion: _root_.scalapb.GeneratedEnumCompanion[ConsoleMessageType] = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType - final def asRecognized: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType.Recognized] = if (isUnrecognized) _root_.scala.None else _root_.scala.Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageType.Recognized]) + def companion: _root_.scalapb.GeneratedEnumCompanion[ConsoleMessageType] = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType + final def asRecognized: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType.Recognized] = if (isUnrecognized) _root_.scala.None else _root_.scala.Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageType.Recognized]) } object ConsoleMessageType extends _root_.scalapb.GeneratedEnumCompanion[ConsoleMessageType] { @@ -57,6 +57,6 @@ object ConsoleMessageType extends _root_.scalapb.GeneratedEnumCompanion[ConsoleM case 3 => DEBUGGER case __other => Unrecognized(__other) } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.EnumDescriptor = ControlcommandsProto.javaDescriptor.getEnumTypes().get(1) - def scalaDescriptor: _root_.scalapb.descriptors.EnumDescriptor = ControlcommandsProto.scalaDescriptor.enums(1) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.EnumDescriptor = ControlCommandsProto.javaDescriptor.getEnumTypes().get(1) + def scalaDescriptor: _root_.scalapb.descriptors.EnumDescriptor = ControlCommandsProto.scalaDescriptor.enums(1) } \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlCommandsProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlCommandsProto.scala new file mode 100644 index 00000000000..a2768f60d43 --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlCommandsProto.scala @@ -0,0 +1,228 @@ +// Generated by the Scala Plugin for the Protocol Buffer Compiler. +// Do not edit! +// +// Protofile syntax: PROTO3 + +package edu.uci.ics.amber.engine.architecture.rpc + +object ControlCommandsProto extends _root_.scalapb.GeneratedFileObject { + lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( + edu.uci.ics.amber.engine.common.VirtualIdentityProto, + edu.uci.ics.amber.engine.common.WorkflowProto, + edu.uci.ics.amber.engine.architecture.worker.StatisticsProto, + edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningsProto, + scalapb.options.ScalapbProto, + com.google.protobuf.timestamp.TimestampProto, + com.google.protobuf.any.AnyProto + ) + lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = + Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( + edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage, + edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, + edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext, + edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation, + edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload, + edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest, + edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest, + edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest, + edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, + edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, + edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest, + edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest, + edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage, + edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest, + edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest, + edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest, + edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest, + edu.uci.ics.amber.engine.architecture.rpc.Ping, + edu.uci.ics.amber.engine.architecture.rpc.Pong, + edu.uci.ics.amber.engine.architecture.rpc.Pass, + edu.uci.ics.amber.engine.architecture.rpc.Nested, + edu.uci.ics.amber.engine.architecture.rpc.MultiCall, + edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand, + edu.uci.ics.amber.engine.architecture.rpc.Collect, + edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber, + edu.uci.ics.amber.engine.architecture.rpc.Chain, + edu.uci.ics.amber.engine.architecture.rpc.Recursion, + edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest, + edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest, + edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest, + edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest, + edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest, + edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest, + edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest, + edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest + ) + private lazy val ProtoBytes: _root_.scala.Array[Byte] = + scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( + """CkBlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy9jb250cm9sX2NvbW1hbmRzLnByb3RvEillZHUud + WNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYxo2ZWR1L3VjaS9pY3MvYW1iZXIvZW5naW5lL2NvbW1vbi92aXJ0d + WFsX2lkZW50aXR5LnByb3RvGi5lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3dvcmtmbG93LnByb3RvGj1lZHUvdWNpL + 2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3dvcmtlci9zdGF0aXN0aWNzLnByb3RvGkdlZHUvdWNpL2ljcy9hbWJlci9lb + mdpbmUvYXJjaGl0ZWN0dXJlL3NlbmRzZW1hbnRpY3MvcGFydGl0aW9uaW5ncy5wcm90bxoVc2NhbGFwYi9zY2FsYXBiLnByb3RvG + h9nb29nbGUvcHJvdG9idWYvdGltZXN0YW1wLnByb3RvGhlnb29nbGUvcHJvdG9idWYvYW55LnByb3RvIq4fCg5Db250cm9sUmVxd + WVzdBK3AQogcHJvcGFnYXRlX2NoYW5uZWxfbWFya2VyX3JlcXVlc3QYASABKAsySC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY + XJjaGl0ZWN0dXJlLnJwYy5Qcm9wYWdhdGVDaGFubmVsTWFya2VyUmVxdWVzdEIi4j8fEh1wcm9wYWdhdGVDaGFubmVsTWFya2VyU + mVxdWVzdEgAUh1wcm9wYWdhdGVDaGFubmVsTWFya2VyUmVxdWVzdBKvAQoedGFrZV9nbG9iYWxfY2hlY2twb2ludF9yZXF1ZXN0G + AIgASgLMkYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuVGFrZUdsb2JhbENoZWNrcG9pbnRSZXF1Z + XN0QiDiPx0SG3Rha2VHbG9iYWxDaGVja3BvaW50UmVxdWVzdEgAUht0YWtlR2xvYmFsQ2hlY2twb2ludFJlcXVlc3QSjgEKFWRlY + nVnX2NvbW1hbmRfcmVxdWVzdBgDIAEoCzI+LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkRlYnVnQ + 29tbWFuZFJlcXVlc3RCGOI/FRITZGVidWdDb21tYW5kUmVxdWVzdEgAUhNkZWJ1Z0NvbW1hbmRSZXF1ZXN0Er8BCiJldmFsdWF0Z + V9weXRob25fZXhwcmVzc2lvbl9yZXF1ZXN0GAQgASgLMkouZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5yc + GMuRXZhbHVhdGVQeXRob25FeHByZXNzaW9uUmVxdWVzdEIk4j8hEh9ldmFsdWF0ZVB5dGhvbkV4cHJlc3Npb25SZXF1ZXN0SABSH + 2V2YWx1YXRlUHl0aG9uRXhwcmVzc2lvblJlcXVlc3QSigEKFG1vZGlmeV9sb2dpY19yZXF1ZXN0GAUgASgLMj0uZWR1LnVjaS5pY + 3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuTW9kaWZ5TG9naWNSZXF1ZXN0QhfiPxQSEm1vZGlmeUxvZ2ljUmVxdWVzd + EgAUhJtb2RpZnlMb2dpY1JlcXVlc3QSkgEKFnJldHJ5X3dvcmtmbG93X3JlcXVlc3QYBiABKAsyPy5lZHUudWNpLmljcy5hbWJlc + i5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5SZXRyeVdvcmtmbG93UmVxdWVzdEIZ4j8WEhRyZXRyeVdvcmtmbG93UmVxdWVzdEgAU + hRyZXRyeVdvcmtmbG93UmVxdWVzdBK7AQohY29uc29sZV9tZXNzYWdlX3RyaWdnZXJlZF9yZXF1ZXN0GAggASgLMkkuZWR1LnVja + S5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuQ29uc29sZU1lc3NhZ2VUcmlnZ2VyZWRSZXF1ZXN0QiPiPyASHmNvb + nNvbGVNZXNzYWdlVHJpZ2dlcmVkUmVxdWVzdEgAUh5jb25zb2xlTWVzc2FnZVRyaWdnZXJlZFJlcXVlc3QSkgEKFnBvcnRfY29tc + GxldGVkX3JlcXVlc3QYCSABKAsyPy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Qb3J0Q29tcGxld + GVkUmVxdWVzdEIZ4j8WEhRwb3J0Q29tcGxldGVkUmVxdWVzdEgAUhRwb3J0Q29tcGxldGVkUmVxdWVzdBKnAQocd29ya2VyX3N0Y + XRlX3VwZGF0ZWRfcmVxdWVzdBgKIAEoCzJELmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLldvcmtlc + lN0YXRlVXBkYXRlZFJlcXVlc3RCHuI/GxIZd29ya2VyU3RhdGVVcGRhdGVkUmVxdWVzdEgAUhl3b3JrZXJTdGF0ZVVwZGF0ZWRSZ + XF1ZXN0EooBChRsaW5rX3dvcmtlcnNfcmVxdWVzdBgLIAEoCzI9LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1c + mUucnBjLkxpbmtXb3JrZXJzUmVxdWVzdEIX4j8UEhJsaW5rV29ya2Vyc1JlcXVlc3RIAFISbGlua1dvcmtlcnNSZXF1ZXN0EpsBC + hlhZGRfaW5wdXRfY2hhbm5lbF9yZXF1ZXN0GDIgASgLMkEuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5yc + GMuQWRkSW5wdXRDaGFubmVsUmVxdWVzdEIb4j8YEhZhZGRJbnB1dENoYW5uZWxSZXF1ZXN0SABSFmFkZElucHV0Q2hhbm5lbFJlc + XVlc3QSmgEKGGFkZF9wYXJ0aXRpb25pbmdfcmVxdWVzdBgzIAEoCzJBLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY + 3R1cmUucnBjLkFkZFBhcnRpdGlvbmluZ1JlcXVlc3RCG+I/GBIWYWRkUGFydGl0aW9uaW5nUmVxdWVzdEgAUhZhZGRQYXJ0aXRpb + 25pbmdSZXF1ZXN0EoYBChNhc3NpZ25fcG9ydF9yZXF1ZXN0GDQgASgLMjwuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpd + GVjdHVyZS5ycGMuQXNzaWduUG9ydFJlcXVlc3RCFuI/ExIRYXNzaWduUG9ydFJlcXVlc3RIAFIRYXNzaWduUG9ydFJlcXVlc3QSp + gEKG2ZpbmFsaXplX2NoZWNrcG9pbnRfcmVxdWVzdBg1IAEoCzJELmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1c + mUucnBjLkZpbmFsaXplQ2hlY2twb2ludFJlcXVlc3RCHuI/GxIZZmluYWxpemVDaGVja3BvaW50UmVxdWVzdEgAUhlmaW5hbGl6Z + UNoZWNrcG9pbnRSZXF1ZXN0EqYBChtpbml0aWFsaXplX2V4ZWN1dG9yX3JlcXVlc3QYNiABKAsyRC5lZHUudWNpLmljcy5hbWJlc + i5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Jbml0aWFsaXplRXhlY3V0b3JSZXF1ZXN0Qh7iPxsSGWluaXRpYWxpemVFeGVjdXRvc + lJlcXVlc3RIAFIZaW5pdGlhbGl6ZUV4ZWN1dG9yUmVxdWVzdBKWAQoXdXBkYXRlX2V4ZWN1dG9yX3JlcXVlc3QYNyABKAsyQC5lZ + HUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5VcGRhdGVFeGVjdXRvclJlcXVlc3RCGuI/FxIVdXBkYXRlR + XhlY3V0b3JSZXF1ZXN0SABSFXVwZGF0ZUV4ZWN1dG9yUmVxdWVzdBJxCg1lbXB0eV9yZXF1ZXN0GDggASgLMjcuZWR1LnVjaS5pY + 3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXF1ZXN0QhHiPw4SDGVtcHR5UmVxdWVzdEgAUgxlbXB0eVJlc + XVlc3QSogEKGnByZXBhcmVfY2hlY2twb2ludF9yZXF1ZXN0GDkgASgLMkMuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpd + GVjdHVyZS5ycGMuUHJlcGFyZUNoZWNrcG9pbnRSZXF1ZXN0Qh3iPxoSGHByZXBhcmVDaGVja3BvaW50UmVxdWVzdEgAUhhwcmVwY + XJlQ2hlY2twb2ludFJlcXVlc3QSmgEKGHF1ZXJ5X3N0YXRpc3RpY3NfcmVxdWVzdBg6IAEoCzJBLmVkdS51Y2kuaWNzLmFtYmVyL + mVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlF1ZXJ5U3RhdGlzdGljc1JlcXVlc3RCG+I/GBIWcXVlcnlTdGF0aXN0aWNzUmVxdWVzd + EgAUhZxdWVyeVN0YXRpc3RpY3NSZXF1ZXN0ElAKBHBpbmcYZCABKAsyLy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0Z + WN0dXJlLnJwYy5QaW5nQgniPwYSBHBpbmdIAFIEcGluZxJQCgRwb25nGGUgASgLMi8uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lL + mFyY2hpdGVjdHVyZS5ycGMuUG9uZ0IJ4j8GEgRwb25nSABSBHBvbmcSWAoGbmVzdGVkGGYgASgLMjEuZWR1LnVjaS5pY3MuYW1iZ + XIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuTmVzdGVkQgviPwgSBm5lc3RlZEgAUgZuZXN0ZWQSUAoEcGFzcxhnIAEoCzIvLmVkd + S51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlBhc3NCCeI/BhIEcGFzc0gAUgRwYXNzEnEKDWVycm9yX2Nvb + W1hbmQYaCABKAsyNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FcnJvckNvbW1hbmRCEeI/DhIMZ + XJyb3JDb21tYW5kSABSDGVycm9yQ29tbWFuZBJkCglyZWN1cnNpb24YaSABKAsyNC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY + XJjaGl0ZWN0dXJlLnJwYy5SZWN1cnNpb25CDuI/CxIJcmVjdXJzaW9uSABSCXJlY3Vyc2lvbhJcCgdjb2xsZWN0GGogASgLMjIuZ + WR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuQ29sbGVjdEIM4j8JEgdjb2xsZWN0SABSB2NvbGxlY3QSe + QoPZ2VuZXJhdGVfbnVtYmVyGGsgASgLMjkuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuR2VuZXJhd + GVOdW1iZXJCE+I/EBIOZ2VuZXJhdGVOdW1iZXJIAFIOZ2VuZXJhdGVOdW1iZXISZQoKbXVsdGlfY2FsbBhsIAEoCzI0LmVkdS51Y + 2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLk11bHRpQ2FsbEIO4j8LEgltdWx0aUNhbGxIAFIJbXVsdGlDYWxsE + lQKBWNoYWluGG0gASgLMjAuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuQ2hhaW5CCuI/BxIFY2hha + W5IAFIFY2hhaW5CDgoMc2VhbGVkX3ZhbHVlIg4KDEVtcHR5UmVxdWVzdCLcAQoPQXN5bmNSUENDb250ZXh0El0KBnNlbmRlchgBI + AEoCzI1LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQWN0b3JWaXJ0dWFsSWRlbnRpdHlCDuI/CxIGc2VuZGVy8AEBU + gZzZW5kZXISYwoIcmVjZWl2ZXIYAiABKAsyNS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkFjdG9yVmlydHVhbElkZ + W50aXR5QhDiPw0SCHJlY2VpdmVy8AEBUghyZWNlaXZlcjoF4j8COAEigAMKEUNvbnRyb2xJbnZvY2F0aW9uEjAKC21ldGhvZF9uY + W1lGAEgASgJQg/iPwwSCm1ldGhvZE5hbWVSCm1ldGhvZE5hbWUSZAoHY29tbWFuZBgCIAEoCzI5LmVkdS51Y2kuaWNzLmFtYmVyL + mVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkNvbnRyb2xSZXF1ZXN0Qg/iPwwSB2NvbW1hbmTwAQFSB2NvbW1hbmQSYgoHY29udGV4d + BgDIAEoCzI6LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkFzeW5jUlBDQ29udGV4dEIM4j8JEgdjb + 250ZXh0Ugdjb250ZXh0Ei0KCmNvbW1hbmRfaWQYBCABKANCDuI/CxIJY29tbWFuZElkUgljb21tYW5kSWQ6QOI/PQo7ZWR1LnVja + S5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5hbWJlcm1lc3NhZ2UuQ29udHJvbFBheWxvYWQiqAUKFENoYW5uZWxNYXJrZXJQYXlsb + 2FkElIKAmlkGAEgASgLMjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5DaGFubmVsTWFya2VySWRlbnRpdHlCCuI/B + xICaWTwAQFSAmlkEm4KC21hcmtlcl90eXBlGAIgASgOMjwuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5yc + GMuQ2hhbm5lbE1hcmtlclR5cGVCD+I/DBIKbWFya2VyVHlwZVIKbWFya2VyVHlwZRJSCgVzY29wZRgDIAMoCzIwLmVkdS51Y2kua + WNzLmFtYmVyLmVuZ2luZS5jb21tb24uQ2hhbm5lbElkZW50aXR5QgriPwcSBXNjb3BlUgVzY29wZRKRAQoPY29tbWFuZF9tYXBwa + W5nGAQgAygLMlMuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuQ2hhbm5lbE1hcmtlclBheWxvYWQuQ + 29tbWFuZE1hcHBpbmdFbnRyeUIT4j8QEg5jb21tYW5kTWFwcGluZ1IOY29tbWFuZE1hcHBpbmcalQEKE0NvbW1hbmRNYXBwaW5nR + W50cnkSGgoDa2V5GAEgASgJQgjiPwUSA2tleVIDa2V5El4KBXZhbHVlGAIgASgLMjwuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lL + mFyY2hpdGVjdHVyZS5ycGMuQ29udHJvbEludm9jYXRpb25CCuI/BxIFdmFsdWVSBXZhbHVlOgI4ATpM4j9JCkdlZHUudWNpLmljc + y5hbWJlci5lbmdpbmUuY29tbW9uLmFtYmVybWVzc2FnZS5Xb3JrZmxvd0ZJRk9NZXNzYWdlUGF5bG9hZCLfBQodUHJvcGFnYXRlQ + 2hhbm5lbE1hcmtlclJlcXVlc3QSgwEKF3NvdXJjZV9vcF90b19zdGFydF9wcm9wGAEgAygLMjMuZWR1LnVjaS5pY3MuYW1iZXIuZ + W5naW5lLmNvbW1vbi5QaHlzaWNhbE9wSWRlbnRpdHlCGOI/FRITc291cmNlT3BUb1N0YXJ0UHJvcFITc291cmNlT3BUb1N0YXJ0U + HJvcBJSCgJpZBgCIAEoCzI2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQ2hhbm5lbE1hcmtlcklkZW50aXR5QgriP + wcSAmlk8AEBUgJpZBJuCgttYXJrZXJfdHlwZRgDIAEoDjI8LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUuc + nBjLkNoYW5uZWxNYXJrZXJUeXBlQg/iPwwSCm1hcmtlclR5cGVSCm1hcmtlclR5cGUSVQoFc2NvcGUYBCADKAsyMy5lZHUudWNpL + mljcy5hbWJlci5lbmdpbmUuY29tbW9uLlBoeXNpY2FsT3BJZGVudGl0eUIK4j8HEgVzY29wZVIFc2NvcGUSYgoKdGFyZ2V0X29wc + xgFIAMoCzIzLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uUGh5c2ljYWxPcElkZW50aXR5Qg7iPwsSCXRhcmdldE9wc + 1IJdGFyZ2V0T3BzEnQKDm1hcmtlcl9jb21tYW5kGAYgASgLMjkuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZ + S5ycGMuQ29udHJvbFJlcXVlc3RCEuI/DxINbWFya2VyQ29tbWFuZFINbWFya2VyQ29tbWFuZBJDChJtYXJrZXJfbWV0aG9kX25hb + WUYByABKAlCFeI/EhIQbWFya2VyTWV0aG9kTmFtZVIQbWFya2VyTWV0aG9kTmFtZSKCAgobVGFrZUdsb2JhbENoZWNrcG9pbnRSZ + XF1ZXN0EjwKD2VzdGltYXRpb25fb25seRgBIAEoCEIT4j8QEg5lc3RpbWF0aW9uT25seVIOZXN0aW1hdGlvbk9ubHkScQoNY2hlY + 2twb2ludF9pZBgCIAEoCzI2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQ2hhbm5lbE1hcmtlcklkZW50aXR5QhTiP + xESDGNoZWNrcG9pbnRJZPABAVIMY2hlY2twb2ludElkEjIKC2Rlc3RpbmF0aW9uGAMgASgJQhDiPw0SC2Rlc3RpbmF0aW9uUgtkZ + XN0aW5hdGlvbiLmAQoaV29ya2Zsb3dSZWNvbmZpZ3VyZVJlcXVlc3QSgAEKD3JlY29uZmlndXJhdGlvbhgBIAEoCzI9LmVkdS51Y + 2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLk1vZGlmeUxvZ2ljUmVxdWVzdEIX4j8UEg9yZWNvbmZpZ3VyYXRpb + 27wAQFSD3JlY29uZmlndXJhdGlvbhJFChJyZWNvbmZpZ3VyYXRpb25faWQYAiABKAlCFuI/ExIRcmVjb25maWd1cmF0aW9uSWRSE + XJlY29uZmlndXJhdGlvbklkIl0KE0RlYnVnQ29tbWFuZFJlcXVlc3QSKgoJd29ya2VyX2lkGAEgASgJQg3iPwoSCHdvcmtlcklkU + gh3b3JrZXJJZBIaCgNjbWQYAiABKAlCCOI/BRIDY21kUgNjbWQihAEKH0V2YWx1YXRlUHl0aG9uRXhwcmVzc2lvblJlcXVlc3QSL + woKZXhwcmVzc2lvbhgBIAEoCUIP4j8MEgpleHByZXNzaW9uUgpleHByZXNzaW9uEjAKC29wZXJhdG9yX2lkGAIgASgJQg/iPwwSC + m9wZXJhdG9ySWRSCm9wZXJhdG9ySWQikQEKEk1vZGlmeUxvZ2ljUmVxdWVzdBJ7Cg51cGRhdGVfcmVxdWVzdBgBIAMoCzJALmVkd + S51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlVwZGF0ZUV4ZWN1dG9yUmVxdWVzdEIS4j8PEg11cGRhdGVSZ + XF1ZXN0Ug11cGRhdGVSZXF1ZXN0InUKFFJldHJ5V29ya2Zsb3dSZXF1ZXN0El0KB3dvcmtlcnMYASADKAsyNS5lZHUudWNpLmljc + y5hbWJlci5lbmdpbmUuY29tbW9uLkFjdG9yVmlydHVhbElkZW50aXR5QgziPwkSB3dvcmtlcnNSB3dvcmtlcnMiowMKDkNvbnNvb + GVNZXNzYWdlEioKCXdvcmtlcl9pZBgBIAEoCUIN4j8KEgh3b3JrZXJJZFIId29ya2VySWQSSwoJdGltZXN0YW1wGAIgASgLMhouZ + 29vZ2xlLnByb3RvYnVmLlRpbWVzdGFtcEIR4j8OEgl0aW1lc3RhbXDwAQFSCXRpbWVzdGFtcBJmCghtc2dfdHlwZRgDIAEoDjI9L + mVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkNvbnNvbGVNZXNzYWdlVHlwZUIM4j8JEgdtc2dUeXBlU + gdtc2dUeXBlEiMKBnNvdXJjZRgEIAEoCUIL4j8IEgZzb3VyY2VSBnNvdXJjZRIgCgV0aXRsZRgFIAEoCUIK4j8HEgV0aXRsZVIFd + Gl0bGUSJgoHbWVzc2FnZRgGIAEoCUIM4j8JEgdtZXNzYWdlUgdtZXNzYWdlOkHiPz4KPGVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZ + S5hcmNoaXRlY3R1cmUuY29udHJvbGxlci5DbGllbnRFdmVudCKcAQoeQ29uc29sZU1lc3NhZ2VUcmlnZ2VyZWRSZXF1ZXN0EnoKD + 2NvbnNvbGVfbWVzc2FnZRgBIAEoCzI5LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkNvbnNvbGVNZ + XNzYWdlQhbiPxMSDmNvbnNvbGVNZXNzYWdl8AEBUg5jb25zb2xlTWVzc2FnZSKQAQoUUG9ydENvbXBsZXRlZFJlcXVlc3QSVgoHc + G9ydF9pZBgBIAEoCzItLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uUG9ydElkZW50aXR5Qg7iPwsSBnBvcnRJZPABA + VIGcG9ydElkEiAKBWlucHV0GAIgASgIQgriPwcSBWlucHV0UgVpbnB1dCJ7ChlXb3JrZXJTdGF0ZVVwZGF0ZWRSZXF1ZXN0El4KB + XN0YXRlGAEgASgOMjkuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS53b3JrZXIuV29ya2VyU3RhdGVCDeI/C + hIFc3RhdGXwAQFSBXN0YXRlImUKEkxpbmtXb3JrZXJzUmVxdWVzdBJPCgRsaW5rGAEgASgLMi0uZWR1LnVjaS5pY3MuYW1iZXIuZ + W5naW5lLmNvbW1vbi5QaHlzaWNhbExpbmtCDOI/CRIEbGlua/ABAVIEbGluayKLAQoEUGluZxIUCgFpGAEgASgFQgbiPwMSAWlSA + WkSGgoDZW5kGAIgASgFQgjiPwUSA2VuZFIDZW5kElEKAnRvGAMgASgLMjUuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vb + i5BY3RvclZpcnR1YWxJZGVudGl0eUIK4j8HEgJ0b/ABAVICdG8iiwEKBFBvbmcSFAoBaRgBIAEoBUIG4j8DEgFpUgFpEhoKA2VuZ + BgCIAEoBUII4j8FEgNlbmRSA2VuZBJRCgJ0bxgDIAEoCzI1LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQWN0b3JWa + XJ0dWFsSWRlbnRpdHlCCuI/BxICdG/wAQFSAnRvIigKBFBhc3MSIAoFdmFsdWUYASABKAlCCuI/BxIFdmFsdWVSBXZhbHVlIh4KB + k5lc3RlZBIUCgFrGAEgASgFQgbiPwMSAWtSAWsiXgoJTXVsdGlDYWxsElEKA3NlcRgBIAMoCzI1LmVkdS51Y2kuaWNzLmFtYmVyL + mVuZ2luZS5jb21tb24uQWN0b3JWaXJ0dWFsSWRlbnRpdHlCCOI/BRIDc2VxUgNzZXEiDgoMRXJyb3JDb21tYW5kImgKB0NvbGxlY + 3QSXQoHd29ya2VycxgBIAMoCzI1LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQWN0b3JWaXJ0dWFsSWRlbnRpdHlCD + OI/CRIHd29ya2Vyc1IHd29ya2VycyIQCg5HZW5lcmF0ZU51bWJlciJgCgVDaGFpbhJXCgVuZXh0cxgBIAMoCzI1LmVkdS51Y2kua + WNzLmFtYmVyLmVuZ2luZS5jb21tb24uQWN0b3JWaXJ0dWFsSWRlbnRpdHlCCuI/BxIFbmV4dHNSBW5leHRzIiEKCVJlY3Vyc2lvb + hIUCgFpGAEgASgFQgbiPwMSAWlSAWki1AEKFkFkZElucHV0Q2hhbm5lbFJlcXVlc3QSYgoKY2hhbm5lbF9pZBgBIAEoCzIwLmVkd + S51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQ2hhbm5lbElkZW50aXR5QhHiPw4SCWNoYW5uZWxJZPABAVIJY2hhbm5lbElkE + lYKB3BvcnRfaWQYAiABKAsyLS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLlBvcnRJZGVudGl0eUIO4j8LEgZwb3J0S + WTwAQFSBnBvcnRJZCLjAQoWQWRkUGFydGl0aW9uaW5nUmVxdWVzdBJMCgN0YWcYASABKAsyLS5lZHUudWNpLmljcy5hbWJlci5lb + mdpbmUuY29tbW9uLlBoeXNpY2FsTGlua0IL4j8IEgN0YWfwAQFSA3RhZxJ7CgxwYXJ0aXRpb25pbmcYAiABKAsyQS5lZHUudWNpL + mljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnNlbmRzZW1hbnRpY3MuUGFydGl0aW9uaW5nQhTiPxESDHBhcnRpdGlvbmluZ + /ABAVIMcGFydGl0aW9uaW5nIs0CChFBc3NpZ25Qb3J0UmVxdWVzdBJWCgdwb3J0X2lkGAEgASgLMi0uZWR1LnVjaS5pY3MuYW1iZ + XIuZW5naW5lLmNvbW1vbi5Qb3J0SWRlbnRpdHlCDuI/CxIGcG9ydElk8AEBUgZwb3J0SWQSIAoFaW5wdXQYAiABKAhCCuI/BxIFa + W5wdXRSBWlucHV0Em0KBnNjaGVtYRgDIAMoCzJILmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkFzc + 2lnblBvcnRSZXF1ZXN0LlNjaGVtYUVudHJ5QgviPwgSBnNjaGVtYVIGc2NoZW1hGk8KC1NjaGVtYUVudHJ5EhoKA2tleRgBIAEoC + UII4j8FEgNrZXlSA2tleRIgCgV2YWx1ZRgCIAEoCUIK4j8HEgV2YWx1ZVIFdmFsdWU6AjgBIrcBChlGaW5hbGl6ZUNoZWNrcG9pb + nRSZXF1ZXN0EnEKDWNoZWNrcG9pbnRfaWQYASABKAsyNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkNoYW5uZWxNY + XJrZXJJZGVudGl0eUIU4j8REgxjaGVja3BvaW50SWTwAQFSDGNoZWNrcG9pbnRJZBInCgh3cml0ZV90bxgCIAEoCUIM4j8JEgd3c + ml0ZVRvUgd3cml0ZVRvIpACChlJbml0aWFsaXplRXhlY3V0b3JSZXF1ZXN0EkMKEnRvdGFsX3dvcmtlcl9jb3VudBgBIAEoBUIV4 + j8SEhB0b3RhbFdvcmtlckNvdW50UhB0b3RhbFdvcmtlckNvdW50ElcKEW9wX2V4ZWNfaW5pdF9pbmZvGAIgASgLMhQuZ29vZ2xlL + nByb3RvYnVmLkFueUIW4j8TEg5vcEV4ZWNJbml0SW5mb/ABAVIOb3BFeGVjSW5pdEluZm8SKgoJaXNfc291cmNlGAMgASgIQg3iP + woSCGlzU291cmNlUghpc1NvdXJjZRIpCghsYW5ndWFnZRgEIAEoCUIN4j8KEghsYW5ndWFnZVIIbGFuZ3VhZ2UirgIKFVVwZGF0Z + UV4ZWN1dG9yUmVxdWVzdBJpCgx0YXJnZXRfb3BfaWQYASABKAsyMy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLlBoe + XNpY2FsT3BJZGVudGl0eUIS4j8PEgp0YXJnZXRPcElk8AEBUgp0YXJnZXRPcElkEkwKDG5ld19leGVjdXRvchgCIAEoCzIULmdvb + 2dsZS5wcm90b2J1Zi5BbnlCE+I/EBILbmV3RXhlY3V0b3LwAQFSC25ld0V4ZWN1dG9yElwKE3N0YXRlX3RyYW5zZmVyX2Z1bmMYA + yABKAsyFC5nb29nbGUucHJvdG9idWYuQW55QhbiPxMSEXN0YXRlVHJhbnNmZXJGdW5jUhFzdGF0ZVRyYW5zZmVyRnVuYyLLAQoYU + HJlcGFyZUNoZWNrcG9pbnRSZXF1ZXN0EnEKDWNoZWNrcG9pbnRfaWQYASABKAsyNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY + 29tbW9uLkNoYW5uZWxNYXJrZXJJZGVudGl0eUIU4j8REgxjaGVja3BvaW50SWTwAQFSDGNoZWNrcG9pbnRJZBI8Cg9lc3RpbWF0a + W9uX29ubHkYAiABKAhCE+I/EBIOZXN0aW1hdGlvbk9ubHlSDmVzdGltYXRpb25Pbmx5IpEBChZRdWVyeVN0YXRpc3RpY3NSZXF1Z + XN0EncKEWZpbHRlcl9ieV93b3JrZXJzGAEgAygLMjUuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5BY3RvclZpcnR1Y + WxJZGVudGl0eUIU4j8REg9maWx0ZXJCeVdvcmtlcnNSD2ZpbHRlckJ5V29ya2VycypnChFDaGFubmVsTWFya2VyVHlwZRItChFSR + VFVSVJFX0FMSUdOTUVOVBAAGhbiPxMSEVJFUVVJUkVfQUxJR05NRU5UEiMKDE5PX0FMSUdOTUVOVBABGhHiPw4SDE5PX0FMSUdOT + UVOVCp6ChJDb25zb2xlTWVzc2FnZVR5cGUSFQoFUFJJTlQQABoK4j8HEgVQUklOVBIVCgVFUlJPUhABGgriPwcSBUVSUk9SEhkKB + 0NPTU1BTkQQAhoM4j8JEgdDT01NQU5EEhsKCERFQlVHR0VSEAMaDeI/ChIIREVCVUdHRVJCC+I/CBABSABYAHgBYgZwcm90bzM=""" + ).mkString) + lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { + val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) + _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) + } + lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { + val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) + com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( + edu.uci.ics.amber.engine.common.VirtualIdentityProto.javaDescriptor, + edu.uci.ics.amber.engine.common.WorkflowProto.javaDescriptor, + edu.uci.ics.amber.engine.architecture.worker.StatisticsProto.javaDescriptor, + edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningsProto.javaDescriptor, + scalapb.options.ScalapbProto.javaDescriptor, + com.google.protobuf.timestamp.TimestampProto.javaDescriptor, + com.google.protobuf.any.AnyProto.javaDescriptor + )) + } + @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") + def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ControlInvocation.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlInvocation.scala similarity index 60% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ControlInvocation.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlInvocation.scala index b1ac6d2416d..68a2f4bceb2 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ControlInvocation.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlInvocation.scala @@ -3,13 +3,13 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlcommands +package edu.uci.ics.amber.engine.architecture.rpc @SerialVersionUID(0L) final case class ControlInvocation( methodName: _root_.scala.Predef.String, - command: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest, - context: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext, + command: edu.uci.ics.amber.engine.architecture.rpc.ControlRequest, + context: edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext, commandId: _root_.scala.Long ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ControlInvocation] with edu.uci.ics.amber.engine.common.ambermessage.ControlPayload { @transient @@ -25,7 +25,7 @@ final case class ControlInvocation( }; { - val __value = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation._typemapper_command.toBase(command) + val __value = edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation._typemapper_command.toBase(command) if (__value.serializedSize != 0) { __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } @@ -63,7 +63,7 @@ final case class ControlInvocation( } }; { - val __v = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation._typemapper_command.toBase(command) + val __v = edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation._typemapper_command.toBase(command) if (__v.serializedSize != 0) { _output__.writeTag(2, 2) _output__.writeUInt32NoTag(__v.serializedSize) @@ -86,8 +86,8 @@ final case class ControlInvocation( }; } def withMethodName(__v: _root_.scala.Predef.String): ControlInvocation = copy(methodName = __v) - def withCommand(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest): ControlInvocation = copy(command = __v) - def withContext(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext): ControlInvocation = copy(context = __v) + def withCommand(__v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequest): ControlInvocation = copy(command = __v) + def withContext(__v: edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext): ControlInvocation = copy(context = __v) def withCommandId(__v: _root_.scala.Long): ControlInvocation = copy(commandId = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { @@ -96,12 +96,12 @@ final case class ControlInvocation( if (__t != "") __t else null } case 2 => { - val __t = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation._typemapper_command.toBase(command) - if (__t != edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.defaultInstance) __t else null + val __t = edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation._typemapper_command.toBase(command) + if (__t != edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.defaultInstance) __t else null } case 3 => { val __t = context - if (__t != edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext.defaultInstance) __t else null } case 4 => { val __t = commandId @@ -113,22 +113,22 @@ final case class ControlInvocation( _root_.scala.Predef.require(__field.containingMessage eq companion.scalaDescriptor) (__field.number: @_root_.scala.unchecked) match { case 1 => _root_.scalapb.descriptors.PString(methodName) - case 2 => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation._typemapper_command.toBase(command).toPMessage + case 2 => edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation._typemapper_command.toBase(command).toPMessage case 3 => context.toPMessage case 4 => _root_.scalapb.descriptors.PLong(commandId) } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation + def companion: edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation.type = edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation]) } -object ControlInvocation extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation = { +object ControlInvocation extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation = { var __methodName: _root_.scala.Predef.String = "" - var __command: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage] = _root_.scala.None - var __context: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext] = _root_.scala.None + var __command: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage] = _root_.scala.None + var __context: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext] = _root_.scala.None var __commandId: _root_.scala.Long = 0L var _done__ = false while (!_done__) { @@ -138,68 +138,68 @@ object ControlInvocation extends scalapb.GeneratedMessageCompanion[edu.uci.ics.a case 10 => __methodName = _input__.readStringRequireUtf8() case 18 => - __command = _root_.scala.Some(__command.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __command = _root_.scala.Some(__command.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 26 => - __context = _root_.scala.Some(__context.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __context = _root_.scala.Some(__context.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 32 => __commandId = _input__.readInt64() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation( + edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation( methodName = __methodName, - command = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation._typemapper_command.toCustom(__command.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.defaultInstance)), - context = __context.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext.defaultInstance), + command = edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation._typemapper_command.toCustom(__command.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.defaultInstance)), + context = __context.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext.defaultInstance), commandId = __commandId ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation( + edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation( methodName = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), - command = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation._typemapper_command.toCustom(__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.defaultInstance)), - context = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext.defaultInstance), + command = edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation._typemapper_command.toCustom(__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.defaultInstance)), + context = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext.defaultInstance), commandId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Long]).getOrElse(0L) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(3) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(3) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(3) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(3) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage - case 3 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext + case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage + case 3 => __out = edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation( methodName = "", - command = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation._typemapper_command.toCustom(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.defaultInstance), - context = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext.defaultInstance, + command = edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation._typemapper_command.toCustom(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.defaultInstance), + context = edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext.defaultInstance, commandId = 0L ) - implicit class ControlInvocationLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation](_l) { + implicit class ControlInvocationLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation](_l) { def methodName: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.methodName)((c_, f_) => c_.copy(methodName = f_)) - def command: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest] = field(_.command)((c_, f_) => c_.copy(command = f_)) - def context: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext] = field(_.context)((c_, f_) => c_.copy(context = f_)) + def command: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlRequest] = field(_.command)((c_, f_) => c_.copy(command = f_)) + def context: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext] = field(_.context)((c_, f_) => c_.copy(context = f_)) def commandId: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.commandId)((c_, f_) => c_.copy(commandId = f_)) } - final val METHODNAME_FIELD_NUMBER = 1 + final val METHOD_NAME_FIELD_NUMBER = 1 final val COMMAND_FIELD_NUMBER = 2 final val CONTEXT_FIELD_NUMBER = 3 - final val COMMANDID_FIELD_NUMBER = 4 + final val COMMAND_ID_FIELD_NUMBER = 4 @transient - private[controlcommands] val _typemapper_command: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest]] + private[rpc] val _typemapper_command: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage, edu.uci.ics.amber.engine.architecture.rpc.ControlRequest] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage, edu.uci.ics.amber.engine.architecture.rpc.ControlRequest]] def of( methodName: _root_.scala.Predef.String, - command: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest, - context: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext, + command: edu.uci.ics.amber.engine.architecture.rpc.ControlRequest, + context: edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext, commandId: _root_.scala.Long - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation = _root_.edu.uci.ics.amber.engine.architecture.rpc.ControlInvocation( methodName, command, context, diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ControlRequest.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlRequest.scala similarity index 59% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ControlRequest.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlRequest.scala index cd0063315ae..5857b892408 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ControlRequest.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlRequest.scala @@ -3,92 +3,92 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlcommands +package edu.uci.ics.amber.engine.architecture.rpc sealed trait ControlRequest extends scalapb.GeneratedSealedOneof { - type MessageType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage - final def isEmpty = this.isInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.Empty.type] + type MessageType = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage + final def isEmpty = this.isInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.Empty.type] final def isDefined = !isEmpty - final def asMessage: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.ControlRequestTypeMapper.toBase(this) - final def asNonEmpty: Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty] = if (isEmpty) None else Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty]) + final def asMessage: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage = edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.ControlRequestTypeMapper.toBase(this) + final def asNonEmpty: Option[edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty] = if (isEmpty) None else Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty]) } object ControlRequest { - case object Empty extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest + case object Empty extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequest - sealed trait NonEmpty extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest - def defaultInstance: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest = Empty + sealed trait NonEmpty extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequest + def defaultInstance: edu.uci.ics.amber.engine.architecture.rpc.ControlRequest = Empty - implicit val ControlRequestTypeMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest] = new _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest] { - override def toCustom(__base: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest = __base.sealedValue match { - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PropagateChannelMarkerRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.TakeGlobalCheckpointRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.DebugCommandRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.EvaluatePythonExpressionRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ModifyLogicRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.RetryWorkflowRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ConsoleMessageTriggeredRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PortCompletedRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.WorkerStateUpdatedRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.LinkWorkersRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AddInputChannelRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AddPartitioningRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AssignPortRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.FinalizeCheckpointRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.InitializeExecutorRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.UpdateExecutorRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.EmptyRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PrepareCheckpointRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.QueryStatisticsRequest => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Ping => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Pong => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Nested => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Pass => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ErrorCommand => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Recursion => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Collect => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.GenerateNumber => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.MultiCall => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Chain => __v.value - case edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Empty => Empty - } - override def toBase(__custom: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage(__custom match { - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PropagateChannelMarkerRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.TakeGlobalCheckpointRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.DebugCommandRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.EvaluatePythonExpressionRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ModifyLogicRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.RetryWorkflowRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ConsoleMessageTriggeredRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PortCompletedRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.WorkerStateUpdatedRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.LinkWorkersRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AddInputChannelRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AddPartitioningRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AssignPortRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.FinalizeCheckpointRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.InitializeExecutorRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.UpdateExecutorRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.EmptyRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PrepareCheckpointRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.QueryStatisticsRequest(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Ping(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Pong(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Nested(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Pass(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ErrorCommand(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Recursion(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Collect(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.GenerateNumber(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.MultiCall(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Chain(__v) - case Empty => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Empty + implicit val ControlRequestTypeMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage, edu.uci.ics.amber.engine.architecture.rpc.ControlRequest] = new _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage, edu.uci.ics.amber.engine.architecture.rpc.ControlRequest] { + override def toCustom(__base: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage): edu.uci.ics.amber.engine.architecture.rpc.ControlRequest = __base.sealedValue match { + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PropagateChannelMarkerRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.TakeGlobalCheckpointRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.DebugCommandRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.EvaluatePythonExpressionRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ModifyLogicRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.RetryWorkflowRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ConsoleMessageTriggeredRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PortCompletedRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.WorkerStateUpdatedRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.LinkWorkersRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AddInputChannelRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AddPartitioningRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AssignPortRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.FinalizeCheckpointRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.InitializeExecutorRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.UpdateExecutorRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.EmptyRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PrepareCheckpointRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.QueryStatisticsRequest => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Ping => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Pong => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Nested => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Pass => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ErrorCommand => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Recursion => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Collect => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.GenerateNumber => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.MultiCall => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Chain => __v.value + case edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Empty => Empty + } + override def toBase(__custom: edu.uci.ics.amber.engine.architecture.rpc.ControlRequest): edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage(__custom match { + case __v: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PropagateChannelMarkerRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.TakeGlobalCheckpointRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.DebugCommandRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.EvaluatePythonExpressionRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ModifyLogicRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.RetryWorkflowRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ConsoleMessageTriggeredRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PortCompletedRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.WorkerStateUpdatedRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.LinkWorkersRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AddInputChannelRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AddPartitioningRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AssignPortRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.FinalizeCheckpointRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.InitializeExecutorRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.UpdateExecutorRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.EmptyRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PrepareCheckpointRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.QueryStatisticsRequest(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.Ping => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Ping(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.Pong => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Pong(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.Nested => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Nested(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.Pass => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Pass(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ErrorCommand(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.Recursion => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Recursion(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.Collect => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Collect(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.GenerateNumber(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.MultiCall => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.MultiCall(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.Chain => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Chain(__v) + case Empty => edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Empty }) } } @SerialVersionUID(0L) final case class ControlRequestMessage( - sealedValue: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue + sealedValue: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ControlRequestMessage] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -397,66 +397,66 @@ final case class ControlRequestMessage( __m.writeTo(_output__) }; } - def getPropagateChannelMarkerRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest = sealedValue.propagateChannelMarkerRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest.defaultInstance) - def withPropagateChannelMarkerRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PropagateChannelMarkerRequest(__v)) - def getTakeGlobalCheckpointRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest = sealedValue.takeGlobalCheckpointRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest.defaultInstance) - def withTakeGlobalCheckpointRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.TakeGlobalCheckpointRequest(__v)) - def getDebugCommandRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest = sealedValue.debugCommandRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest.defaultInstance) - def withDebugCommandRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.DebugCommandRequest(__v)) - def getEvaluatePythonExpressionRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest = sealedValue.evaluatePythonExpressionRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest.defaultInstance) - def withEvaluatePythonExpressionRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.EvaluatePythonExpressionRequest(__v)) - def getModifyLogicRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest = sealedValue.modifyLogicRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest.defaultInstance) - def withModifyLogicRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ModifyLogicRequest(__v)) - def getRetryWorkflowRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest = sealedValue.retryWorkflowRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest.defaultInstance) - def withRetryWorkflowRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.RetryWorkflowRequest(__v)) - def getConsoleMessageTriggeredRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest = sealedValue.consoleMessageTriggeredRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest.defaultInstance) - def withConsoleMessageTriggeredRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ConsoleMessageTriggeredRequest(__v)) - def getPortCompletedRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest = sealedValue.portCompletedRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest.defaultInstance) - def withPortCompletedRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PortCompletedRequest(__v)) - def getWorkerStateUpdatedRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest = sealedValue.workerStateUpdatedRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest.defaultInstance) - def withWorkerStateUpdatedRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.WorkerStateUpdatedRequest(__v)) - def getLinkWorkersRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest = sealedValue.linkWorkersRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest.defaultInstance) - def withLinkWorkersRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.LinkWorkersRequest(__v)) - def getAddInputChannelRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest = sealedValue.addInputChannelRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest.defaultInstance) - def withAddInputChannelRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AddInputChannelRequest(__v)) - def getAddPartitioningRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest = sealedValue.addPartitioningRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest.defaultInstance) - def withAddPartitioningRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AddPartitioningRequest(__v)) - def getAssignPortRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest = sealedValue.assignPortRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.defaultInstance) - def withAssignPortRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AssignPortRequest(__v)) - def getFinalizeCheckpointRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest = sealedValue.finalizeCheckpointRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest.defaultInstance) - def withFinalizeCheckpointRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.FinalizeCheckpointRequest(__v)) - def getInitializeExecutorRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest = sealedValue.initializeExecutorRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest.defaultInstance) - def withInitializeExecutorRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.InitializeExecutorRequest(__v)) - def getUpdateExecutorRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest = sealedValue.updateExecutorRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest.defaultInstance) - def withUpdateExecutorRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.UpdateExecutorRequest(__v)) - def getEmptyRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest = sealedValue.emptyRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest.defaultInstance) - def withEmptyRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.EmptyRequest(__v)) - def getPrepareCheckpointRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest = sealedValue.prepareCheckpointRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest.defaultInstance) - def withPrepareCheckpointRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PrepareCheckpointRequest(__v)) - def getQueryStatisticsRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest = sealedValue.queryStatisticsRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest.defaultInstance) - def withQueryStatisticsRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.QueryStatisticsRequest(__v)) - def getPing: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping = sealedValue.ping.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping.defaultInstance) - def withPing(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Ping(__v)) - def getPong: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong = sealedValue.pong.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong.defaultInstance) - def withPong(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Pong(__v)) - def getNested: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested = sealedValue.nested.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested.defaultInstance) - def withNested(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Nested(__v)) - def getPass: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass = sealedValue.pass.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass.defaultInstance) - def withPass(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Pass(__v)) - def getErrorCommand: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand = sealedValue.errorCommand.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand.defaultInstance) - def withErrorCommand(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ErrorCommand(__v)) - def getRecursion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion = sealedValue.recursion.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion.defaultInstance) - def withRecursion(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Recursion(__v)) - def getCollect: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect = sealedValue.collect.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect.defaultInstance) - def withCollect(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Collect(__v)) - def getGenerateNumber: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber = sealedValue.generateNumber.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber.defaultInstance) - def withGenerateNumber(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.GenerateNumber(__v)) - def getMultiCall: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall = sealedValue.multiCall.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall.defaultInstance) - def withMultiCall(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.MultiCall(__v)) - def getChain: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain = sealedValue.chain.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain.defaultInstance) - def withChain(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Chain(__v)) - def clearSealedValue: ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Empty) - def withSealedValue(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue): ControlRequestMessage = copy(sealedValue = __v) + def getPropagateChannelMarkerRequest: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest = sealedValue.propagateChannelMarkerRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest.defaultInstance) + def withPropagateChannelMarkerRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PropagateChannelMarkerRequest(__v)) + def getTakeGlobalCheckpointRequest: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest = sealedValue.takeGlobalCheckpointRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest.defaultInstance) + def withTakeGlobalCheckpointRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.TakeGlobalCheckpointRequest(__v)) + def getDebugCommandRequest: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest = sealedValue.debugCommandRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest.defaultInstance) + def withDebugCommandRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.DebugCommandRequest(__v)) + def getEvaluatePythonExpressionRequest: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest = sealedValue.evaluatePythonExpressionRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest.defaultInstance) + def withEvaluatePythonExpressionRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.EvaluatePythonExpressionRequest(__v)) + def getModifyLogicRequest: edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest = sealedValue.modifyLogicRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest.defaultInstance) + def withModifyLogicRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ModifyLogicRequest(__v)) + def getRetryWorkflowRequest: edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest = sealedValue.retryWorkflowRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest.defaultInstance) + def withRetryWorkflowRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.RetryWorkflowRequest(__v)) + def getConsoleMessageTriggeredRequest: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest = sealedValue.consoleMessageTriggeredRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest.defaultInstance) + def withConsoleMessageTriggeredRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ConsoleMessageTriggeredRequest(__v)) + def getPortCompletedRequest: edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest = sealedValue.portCompletedRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest.defaultInstance) + def withPortCompletedRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PortCompletedRequest(__v)) + def getWorkerStateUpdatedRequest: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest = sealedValue.workerStateUpdatedRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest.defaultInstance) + def withWorkerStateUpdatedRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.WorkerStateUpdatedRequest(__v)) + def getLinkWorkersRequest: edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest = sealedValue.linkWorkersRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest.defaultInstance) + def withLinkWorkersRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.LinkWorkersRequest(__v)) + def getAddInputChannelRequest: edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest = sealedValue.addInputChannelRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest.defaultInstance) + def withAddInputChannelRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AddInputChannelRequest(__v)) + def getAddPartitioningRequest: edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest = sealedValue.addPartitioningRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest.defaultInstance) + def withAddPartitioningRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AddPartitioningRequest(__v)) + def getAssignPortRequest: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest = sealedValue.assignPortRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.defaultInstance) + def withAssignPortRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AssignPortRequest(__v)) + def getFinalizeCheckpointRequest: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest = sealedValue.finalizeCheckpointRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest.defaultInstance) + def withFinalizeCheckpointRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.FinalizeCheckpointRequest(__v)) + def getInitializeExecutorRequest: edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest = sealedValue.initializeExecutorRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest.defaultInstance) + def withInitializeExecutorRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.InitializeExecutorRequest(__v)) + def getUpdateExecutorRequest: edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest = sealedValue.updateExecutorRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest.defaultInstance) + def withUpdateExecutorRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.UpdateExecutorRequest(__v)) + def getEmptyRequest: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest = sealedValue.emptyRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest.defaultInstance) + def withEmptyRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.EmptyRequest(__v)) + def getPrepareCheckpointRequest: edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest = sealedValue.prepareCheckpointRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest.defaultInstance) + def withPrepareCheckpointRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PrepareCheckpointRequest(__v)) + def getQueryStatisticsRequest: edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest = sealedValue.queryStatisticsRequest.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest.defaultInstance) + def withQueryStatisticsRequest(__v: edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.QueryStatisticsRequest(__v)) + def getPing: edu.uci.ics.amber.engine.architecture.rpc.Ping = sealedValue.ping.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.Ping.defaultInstance) + def withPing(__v: edu.uci.ics.amber.engine.architecture.rpc.Ping): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Ping(__v)) + def getPong: edu.uci.ics.amber.engine.architecture.rpc.Pong = sealedValue.pong.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.Pong.defaultInstance) + def withPong(__v: edu.uci.ics.amber.engine.architecture.rpc.Pong): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Pong(__v)) + def getNested: edu.uci.ics.amber.engine.architecture.rpc.Nested = sealedValue.nested.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.Nested.defaultInstance) + def withNested(__v: edu.uci.ics.amber.engine.architecture.rpc.Nested): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Nested(__v)) + def getPass: edu.uci.ics.amber.engine.architecture.rpc.Pass = sealedValue.pass.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.Pass.defaultInstance) + def withPass(__v: edu.uci.ics.amber.engine.architecture.rpc.Pass): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Pass(__v)) + def getErrorCommand: edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand = sealedValue.errorCommand.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand.defaultInstance) + def withErrorCommand(__v: edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ErrorCommand(__v)) + def getRecursion: edu.uci.ics.amber.engine.architecture.rpc.Recursion = sealedValue.recursion.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.Recursion.defaultInstance) + def withRecursion(__v: edu.uci.ics.amber.engine.architecture.rpc.Recursion): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Recursion(__v)) + def getCollect: edu.uci.ics.amber.engine.architecture.rpc.Collect = sealedValue.collect.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.Collect.defaultInstance) + def withCollect(__v: edu.uci.ics.amber.engine.architecture.rpc.Collect): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Collect(__v)) + def getGenerateNumber: edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber = sealedValue.generateNumber.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber.defaultInstance) + def withGenerateNumber(__v: edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.GenerateNumber(__v)) + def getMultiCall: edu.uci.ics.amber.engine.architecture.rpc.MultiCall = sealedValue.multiCall.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.MultiCall.defaultInstance) + def withMultiCall(__v: edu.uci.ics.amber.engine.architecture.rpc.MultiCall): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.MultiCall(__v)) + def getChain: edu.uci.ics.amber.engine.architecture.rpc.Chain = sealedValue.chain.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.Chain.defaultInstance) + def withChain(__v: edu.uci.ics.amber.engine.architecture.rpc.Chain): ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Chain(__v)) + def clearSealedValue: ControlRequestMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Empty) + def withSealedValue(__v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue): ControlRequestMessage = copy(sealedValue = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => sealedValue.propagateChannelMarkerRequest.orNull @@ -525,163 +525,163 @@ final case class ControlRequestMessage( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage - def toControlRequest: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.ControlRequestTypeMapper.toCustom(this) + def companion: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.type = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage + def toControlRequest: edu.uci.ics.amber.engine.architecture.rpc.ControlRequest = edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.ControlRequestTypeMapper.toCustom(this) // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.ControlRequest]) } -object ControlRequestMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage = { - var __sealedValue: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Empty +object ControlRequestMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage = { + var __sealedValue: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Empty var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PropagateChannelMarkerRequest(__sealedValue.propagateChannelMarkerRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PropagateChannelMarkerRequest(__sealedValue.propagateChannelMarkerRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.TakeGlobalCheckpointRequest(__sealedValue.takeGlobalCheckpointRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.TakeGlobalCheckpointRequest(__sealedValue.takeGlobalCheckpointRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 26 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.DebugCommandRequest(__sealedValue.debugCommandRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.DebugCommandRequest(__sealedValue.debugCommandRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 34 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.EvaluatePythonExpressionRequest(__sealedValue.evaluatePythonExpressionRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.EvaluatePythonExpressionRequest(__sealedValue.evaluatePythonExpressionRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 42 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ModifyLogicRequest(__sealedValue.modifyLogicRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ModifyLogicRequest(__sealedValue.modifyLogicRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 50 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.RetryWorkflowRequest(__sealedValue.retryWorkflowRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.RetryWorkflowRequest(__sealedValue.retryWorkflowRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 66 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ConsoleMessageTriggeredRequest(__sealedValue.consoleMessageTriggeredRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ConsoleMessageTriggeredRequest(__sealedValue.consoleMessageTriggeredRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 74 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PortCompletedRequest(__sealedValue.portCompletedRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PortCompletedRequest(__sealedValue.portCompletedRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 82 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.WorkerStateUpdatedRequest(__sealedValue.workerStateUpdatedRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.WorkerStateUpdatedRequest(__sealedValue.workerStateUpdatedRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 90 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.LinkWorkersRequest(__sealedValue.linkWorkersRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.LinkWorkersRequest(__sealedValue.linkWorkersRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 402 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AddInputChannelRequest(__sealedValue.addInputChannelRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AddInputChannelRequest(__sealedValue.addInputChannelRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 410 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AddPartitioningRequest(__sealedValue.addPartitioningRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AddPartitioningRequest(__sealedValue.addPartitioningRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 418 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AssignPortRequest(__sealedValue.assignPortRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AssignPortRequest(__sealedValue.assignPortRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 426 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.FinalizeCheckpointRequest(__sealedValue.finalizeCheckpointRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.FinalizeCheckpointRequest(__sealedValue.finalizeCheckpointRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 434 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.InitializeExecutorRequest(__sealedValue.initializeExecutorRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.InitializeExecutorRequest(__sealedValue.initializeExecutorRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 442 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.UpdateExecutorRequest(__sealedValue.updateExecutorRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.UpdateExecutorRequest(__sealedValue.updateExecutorRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 450 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.EmptyRequest(__sealedValue.emptyRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.EmptyRequest(__sealedValue.emptyRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 458 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PrepareCheckpointRequest(__sealedValue.prepareCheckpointRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PrepareCheckpointRequest(__sealedValue.prepareCheckpointRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 466 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.QueryStatisticsRequest(__sealedValue.queryStatisticsRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.QueryStatisticsRequest(__sealedValue.queryStatisticsRequest.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 802 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Ping(__sealedValue.ping.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Ping(__sealedValue.ping.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.Ping](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 810 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Pong(__sealedValue.pong.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Pong(__sealedValue.pong.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.Pong](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 818 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Nested(__sealedValue.nested.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Nested(__sealedValue.nested.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.Nested](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 826 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Pass(__sealedValue.pass.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Pass(__sealedValue.pass.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.Pass](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 834 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ErrorCommand(__sealedValue.errorCommand.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ErrorCommand(__sealedValue.errorCommand.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 842 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Recursion(__sealedValue.recursion.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Recursion(__sealedValue.recursion.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.Recursion](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 850 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Collect(__sealedValue.collect.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Collect(__sealedValue.collect.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.Collect](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 858 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.GenerateNumber(__sealedValue.generateNumber.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.GenerateNumber(__sealedValue.generateNumber.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 866 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.MultiCall(__sealedValue.multiCall.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.MultiCall(__sealedValue.multiCall.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.MultiCall](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 874 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Chain(__sealedValue.chain.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Chain(__sealedValue.chain.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.Chain](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage( + edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage( sealedValue = __sealedValue ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage( - sealedValue = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PropagateChannelMarkerRequest(_)) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.TakeGlobalCheckpointRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.DebugCommandRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.EvaluatePythonExpressionRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(5).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ModifyLogicRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(6).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.RetryWorkflowRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(8).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ConsoleMessageTriggeredRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(9).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PortCompletedRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(10).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.WorkerStateUpdatedRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(11).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.LinkWorkersRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(50).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AddInputChannelRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(51).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AddPartitioningRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(52).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AssignPortRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(53).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.FinalizeCheckpointRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(54).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.InitializeExecutorRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(55).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.UpdateExecutorRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(56).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.EmptyRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(57).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PrepareCheckpointRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(58).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.QueryStatisticsRequest(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(100).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Ping(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(101).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Pong(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(102).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Nested(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(103).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Pass(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(104).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ErrorCommand(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(105).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Recursion(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(106).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Collect(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(107).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.GenerateNumber(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(108).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.MultiCall(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(109).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Chain(_))) - .getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Empty) + edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage( + sealedValue = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PropagateChannelMarkerRequest(_)) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.TakeGlobalCheckpointRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.DebugCommandRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.EvaluatePythonExpressionRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(5).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ModifyLogicRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(6).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.RetryWorkflowRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(8).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ConsoleMessageTriggeredRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(9).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PortCompletedRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(10).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.WorkerStateUpdatedRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(11).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.LinkWorkersRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(50).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AddInputChannelRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(51).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AddPartitioningRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(52).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AssignPortRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(53).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.FinalizeCheckpointRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(54).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.InitializeExecutorRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(55).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.UpdateExecutorRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(56).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.EmptyRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(57).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PrepareCheckpointRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(58).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.QueryStatisticsRequest(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(100).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Ping]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Ping(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(101).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Pong]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Pong(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(102).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Nested]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Nested(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(103).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Pass]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Pass(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(104).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ErrorCommand(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(105).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Recursion]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Recursion(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(106).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Collect]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Collect(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(107).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.GenerateNumber(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(108).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.MultiCall]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.MultiCall(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(109).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Chain]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Chain(_))) + .getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest - case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest - case 3 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest - case 4 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest - case 5 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest - case 6 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest - case 8 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest - case 9 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest - case 10 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest - case 11 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest - case 50 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest - case 51 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest - case 52 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest - case 53 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest - case 54 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest - case 55 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest - case 56 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest - case 57 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest - case 58 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest - case 100 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping - case 101 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong - case 102 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested - case 103 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass - case 104 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand - case 105 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion - case 106 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect - case 107 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber - case 108 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall - case 109 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain + case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest + case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest + case 3 => __out = edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest + case 4 => __out = edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest + case 5 => __out = edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest + case 6 => __out = edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest + case 8 => __out = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest + case 9 => __out = edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest + case 10 => __out = edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest + case 11 => __out = edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest + case 50 => __out = edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest + case 51 => __out = edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest + case 52 => __out = edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest + case 53 => __out = edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest + case 54 => __out = edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest + case 55 => __out = edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest + case 56 => __out = edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest + case 57 => __out = edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest + case 58 => __out = edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest + case 100 => __out = edu.uci.ics.amber.engine.architecture.rpc.Ping + case 101 => __out = edu.uci.ics.amber.engine.architecture.rpc.Pong + case 102 => __out = edu.uci.ics.amber.engine.architecture.rpc.Nested + case 103 => __out = edu.uci.ics.amber.engine.architecture.rpc.Pass + case 104 => __out = edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand + case 105 => __out = edu.uci.ics.amber.engine.architecture.rpc.Recursion + case 106 => __out = edu.uci.ics.amber.engine.architecture.rpc.Collect + case 107 => __out = edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber + case 108 => __out = edu.uci.ics.amber.engine.architecture.rpc.MultiCall + case 109 => __out = edu.uci.ics.amber.engine.architecture.rpc.Chain } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage( - sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Empty + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage( + sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Empty ) sealed trait SealedValue extends _root_.scalapb.GeneratedOneof { def isEmpty: _root_.scala.Boolean = false @@ -715,39 +715,39 @@ object ControlRequestMessage extends scalapb.GeneratedMessageCompanion[edu.uci.i def isGenerateNumber: _root_.scala.Boolean = false def isMultiCall: _root_.scala.Boolean = false def isChain: _root_.scala.Boolean = false - def propagateChannelMarkerRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest] = _root_.scala.None - def takeGlobalCheckpointRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest] = _root_.scala.None - def debugCommandRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest] = _root_.scala.None - def evaluatePythonExpressionRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest] = _root_.scala.None - def modifyLogicRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest] = _root_.scala.None - def retryWorkflowRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest] = _root_.scala.None - def consoleMessageTriggeredRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest] = _root_.scala.None - def portCompletedRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest] = _root_.scala.None - def workerStateUpdatedRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest] = _root_.scala.None - def linkWorkersRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest] = _root_.scala.None - def addInputChannelRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest] = _root_.scala.None - def addPartitioningRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest] = _root_.scala.None - def assignPortRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest] = _root_.scala.None - def finalizeCheckpointRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest] = _root_.scala.None - def initializeExecutorRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest] = _root_.scala.None - def updateExecutorRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest] = _root_.scala.None - def emptyRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest] = _root_.scala.None - def prepareCheckpointRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest] = _root_.scala.None - def queryStatisticsRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest] = _root_.scala.None - def ping: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping] = _root_.scala.None - def pong: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong] = _root_.scala.None - def nested: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested] = _root_.scala.None - def pass: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass] = _root_.scala.None - def errorCommand: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand] = _root_.scala.None - def recursion: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion] = _root_.scala.None - def collect: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect] = _root_.scala.None - def generateNumber: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber] = _root_.scala.None - def multiCall: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall] = _root_.scala.None - def chain: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain] = _root_.scala.None + def propagateChannelMarkerRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest] = _root_.scala.None + def takeGlobalCheckpointRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest] = _root_.scala.None + def debugCommandRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest] = _root_.scala.None + def evaluatePythonExpressionRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest] = _root_.scala.None + def modifyLogicRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest] = _root_.scala.None + def retryWorkflowRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest] = _root_.scala.None + def consoleMessageTriggeredRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest] = _root_.scala.None + def portCompletedRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest] = _root_.scala.None + def workerStateUpdatedRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest] = _root_.scala.None + def linkWorkersRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest] = _root_.scala.None + def addInputChannelRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest] = _root_.scala.None + def addPartitioningRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest] = _root_.scala.None + def assignPortRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest] = _root_.scala.None + def finalizeCheckpointRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest] = _root_.scala.None + def initializeExecutorRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest] = _root_.scala.None + def updateExecutorRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest] = _root_.scala.None + def emptyRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest] = _root_.scala.None + def prepareCheckpointRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest] = _root_.scala.None + def queryStatisticsRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest] = _root_.scala.None + def ping: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Ping] = _root_.scala.None + def pong: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Pong] = _root_.scala.None + def nested: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Nested] = _root_.scala.None + def pass: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Pass] = _root_.scala.None + def errorCommand: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand] = _root_.scala.None + def recursion: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Recursion] = _root_.scala.None + def collect: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Collect] = _root_.scala.None + def generateNumber: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber] = _root_.scala.None + def multiCall: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.MultiCall] = _root_.scala.None + def chain: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Chain] = _root_.scala.None } object SealedValue { @SerialVersionUID(0L) - case object Empty extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { + case object Empty extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { type ValueType = _root_.scala.Nothing override def isEmpty: _root_.scala.Boolean = true override def isDefined: _root_.scala.Boolean = false @@ -756,273 +756,273 @@ object ControlRequestMessage extends scalapb.GeneratedMessageCompanion[edu.uci.i } @SerialVersionUID(0L) - final case class PropagateChannelMarkerRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest + final case class PropagateChannelMarkerRequest(value: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest override def isPropagateChannelMarkerRequest: _root_.scala.Boolean = true - override def propagateChannelMarkerRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest] = Some(value) + override def propagateChannelMarkerRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest] = Some(value) override def number: _root_.scala.Int = 1 } @SerialVersionUID(0L) - final case class TakeGlobalCheckpointRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest + final case class TakeGlobalCheckpointRequest(value: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest override def isTakeGlobalCheckpointRequest: _root_.scala.Boolean = true - override def takeGlobalCheckpointRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest] = Some(value) + override def takeGlobalCheckpointRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest] = Some(value) override def number: _root_.scala.Int = 2 } @SerialVersionUID(0L) - final case class DebugCommandRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest + final case class DebugCommandRequest(value: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest override def isDebugCommandRequest: _root_.scala.Boolean = true - override def debugCommandRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest] = Some(value) + override def debugCommandRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest] = Some(value) override def number: _root_.scala.Int = 3 } @SerialVersionUID(0L) - final case class EvaluatePythonExpressionRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest + final case class EvaluatePythonExpressionRequest(value: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest override def isEvaluatePythonExpressionRequest: _root_.scala.Boolean = true - override def evaluatePythonExpressionRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest] = Some(value) + override def evaluatePythonExpressionRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest] = Some(value) override def number: _root_.scala.Int = 4 } @SerialVersionUID(0L) - final case class ModifyLogicRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest + final case class ModifyLogicRequest(value: edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest override def isModifyLogicRequest: _root_.scala.Boolean = true - override def modifyLogicRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest] = Some(value) + override def modifyLogicRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest] = Some(value) override def number: _root_.scala.Int = 5 } @SerialVersionUID(0L) - final case class RetryWorkflowRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest + final case class RetryWorkflowRequest(value: edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest override def isRetryWorkflowRequest: _root_.scala.Boolean = true - override def retryWorkflowRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest] = Some(value) + override def retryWorkflowRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest] = Some(value) override def number: _root_.scala.Int = 6 } @SerialVersionUID(0L) - final case class ConsoleMessageTriggeredRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest + final case class ConsoleMessageTriggeredRequest(value: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest override def isConsoleMessageTriggeredRequest: _root_.scala.Boolean = true - override def consoleMessageTriggeredRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest] = Some(value) + override def consoleMessageTriggeredRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest] = Some(value) override def number: _root_.scala.Int = 8 } @SerialVersionUID(0L) - final case class PortCompletedRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest + final case class PortCompletedRequest(value: edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest override def isPortCompletedRequest: _root_.scala.Boolean = true - override def portCompletedRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest] = Some(value) + override def portCompletedRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest] = Some(value) override def number: _root_.scala.Int = 9 } @SerialVersionUID(0L) - final case class WorkerStateUpdatedRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest + final case class WorkerStateUpdatedRequest(value: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest override def isWorkerStateUpdatedRequest: _root_.scala.Boolean = true - override def workerStateUpdatedRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest] = Some(value) + override def workerStateUpdatedRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest] = Some(value) override def number: _root_.scala.Int = 10 } @SerialVersionUID(0L) - final case class LinkWorkersRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest + final case class LinkWorkersRequest(value: edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest override def isLinkWorkersRequest: _root_.scala.Boolean = true - override def linkWorkersRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest] = Some(value) + override def linkWorkersRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest] = Some(value) override def number: _root_.scala.Int = 11 } @SerialVersionUID(0L) - final case class AddInputChannelRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest + final case class AddInputChannelRequest(value: edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest override def isAddInputChannelRequest: _root_.scala.Boolean = true - override def addInputChannelRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest] = Some(value) + override def addInputChannelRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest] = Some(value) override def number: _root_.scala.Int = 50 } @SerialVersionUID(0L) - final case class AddPartitioningRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest + final case class AddPartitioningRequest(value: edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest override def isAddPartitioningRequest: _root_.scala.Boolean = true - override def addPartitioningRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest] = Some(value) + override def addPartitioningRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest] = Some(value) override def number: _root_.scala.Int = 51 } @SerialVersionUID(0L) - final case class AssignPortRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest + final case class AssignPortRequest(value: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest override def isAssignPortRequest: _root_.scala.Boolean = true - override def assignPortRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest] = Some(value) + override def assignPortRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest] = Some(value) override def number: _root_.scala.Int = 52 } @SerialVersionUID(0L) - final case class FinalizeCheckpointRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest + final case class FinalizeCheckpointRequest(value: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest override def isFinalizeCheckpointRequest: _root_.scala.Boolean = true - override def finalizeCheckpointRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest] = Some(value) + override def finalizeCheckpointRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest] = Some(value) override def number: _root_.scala.Int = 53 } @SerialVersionUID(0L) - final case class InitializeExecutorRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest + final case class InitializeExecutorRequest(value: edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest override def isInitializeExecutorRequest: _root_.scala.Boolean = true - override def initializeExecutorRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest] = Some(value) + override def initializeExecutorRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest] = Some(value) override def number: _root_.scala.Int = 54 } @SerialVersionUID(0L) - final case class UpdateExecutorRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest + final case class UpdateExecutorRequest(value: edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest override def isUpdateExecutorRequest: _root_.scala.Boolean = true - override def updateExecutorRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest] = Some(value) + override def updateExecutorRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest] = Some(value) override def number: _root_.scala.Int = 55 } @SerialVersionUID(0L) - final case class EmptyRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest + final case class EmptyRequest(value: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest override def isEmptyRequest: _root_.scala.Boolean = true - override def emptyRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest] = Some(value) + override def emptyRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest] = Some(value) override def number: _root_.scala.Int = 56 } @SerialVersionUID(0L) - final case class PrepareCheckpointRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest + final case class PrepareCheckpointRequest(value: edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest override def isPrepareCheckpointRequest: _root_.scala.Boolean = true - override def prepareCheckpointRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest] = Some(value) + override def prepareCheckpointRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest] = Some(value) override def number: _root_.scala.Int = 57 } @SerialVersionUID(0L) - final case class QueryStatisticsRequest(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest + final case class QueryStatisticsRequest(value: edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest override def isQueryStatisticsRequest: _root_.scala.Boolean = true - override def queryStatisticsRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest] = Some(value) + override def queryStatisticsRequest: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest] = Some(value) override def number: _root_.scala.Int = 58 } @SerialVersionUID(0L) - final case class Ping(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping + final case class Ping(value: edu.uci.ics.amber.engine.architecture.rpc.Ping) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.Ping override def isPing: _root_.scala.Boolean = true - override def ping: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping] = Some(value) + override def ping: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Ping] = Some(value) override def number: _root_.scala.Int = 100 } @SerialVersionUID(0L) - final case class Pong(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong + final case class Pong(value: edu.uci.ics.amber.engine.architecture.rpc.Pong) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.Pong override def isPong: _root_.scala.Boolean = true - override def pong: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong] = Some(value) + override def pong: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Pong] = Some(value) override def number: _root_.scala.Int = 101 } @SerialVersionUID(0L) - final case class Nested(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested + final case class Nested(value: edu.uci.ics.amber.engine.architecture.rpc.Nested) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.Nested override def isNested: _root_.scala.Boolean = true - override def nested: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested] = Some(value) + override def nested: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Nested] = Some(value) override def number: _root_.scala.Int = 102 } @SerialVersionUID(0L) - final case class Pass(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass + final case class Pass(value: edu.uci.ics.amber.engine.architecture.rpc.Pass) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.Pass override def isPass: _root_.scala.Boolean = true - override def pass: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass] = Some(value) + override def pass: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Pass] = Some(value) override def number: _root_.scala.Int = 103 } @SerialVersionUID(0L) - final case class ErrorCommand(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand + final case class ErrorCommand(value: edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand override def isErrorCommand: _root_.scala.Boolean = true - override def errorCommand: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand] = Some(value) + override def errorCommand: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand] = Some(value) override def number: _root_.scala.Int = 104 } @SerialVersionUID(0L) - final case class Recursion(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion + final case class Recursion(value: edu.uci.ics.amber.engine.architecture.rpc.Recursion) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.Recursion override def isRecursion: _root_.scala.Boolean = true - override def recursion: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion] = Some(value) + override def recursion: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Recursion] = Some(value) override def number: _root_.scala.Int = 105 } @SerialVersionUID(0L) - final case class Collect(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect + final case class Collect(value: edu.uci.ics.amber.engine.architecture.rpc.Collect) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.Collect override def isCollect: _root_.scala.Boolean = true - override def collect: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect] = Some(value) + override def collect: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Collect] = Some(value) override def number: _root_.scala.Int = 106 } @SerialVersionUID(0L) - final case class GenerateNumber(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber + final case class GenerateNumber(value: edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber override def isGenerateNumber: _root_.scala.Boolean = true - override def generateNumber: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber] = Some(value) + override def generateNumber: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber] = Some(value) override def number: _root_.scala.Int = 107 } @SerialVersionUID(0L) - final case class MultiCall(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall + final case class MultiCall(value: edu.uci.ics.amber.engine.architecture.rpc.MultiCall) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.MultiCall override def isMultiCall: _root_.scala.Boolean = true - override def multiCall: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall] = Some(value) + override def multiCall: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.MultiCall] = Some(value) override def number: _root_.scala.Int = 108 } @SerialVersionUID(0L) - final case class Chain(value: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain) extends edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain + final case class Chain(value: edu.uci.ics.amber.engine.architecture.rpc.Chain) extends edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.Chain override def isChain: _root_.scala.Boolean = true - override def chain: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain] = Some(value) + override def chain: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.Chain] = Some(value) override def number: _root_.scala.Int = 109 } } - implicit class ControlRequestMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage](_l) { - def propagateChannelMarkerRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest] = field(_.getPropagateChannelMarkerRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PropagateChannelMarkerRequest(f_))) - def takeGlobalCheckpointRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest] = field(_.getTakeGlobalCheckpointRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.TakeGlobalCheckpointRequest(f_))) - def debugCommandRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest] = field(_.getDebugCommandRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.DebugCommandRequest(f_))) - def evaluatePythonExpressionRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest] = field(_.getEvaluatePythonExpressionRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.EvaluatePythonExpressionRequest(f_))) - def modifyLogicRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest] = field(_.getModifyLogicRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ModifyLogicRequest(f_))) - def retryWorkflowRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest] = field(_.getRetryWorkflowRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.RetryWorkflowRequest(f_))) - def consoleMessageTriggeredRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest] = field(_.getConsoleMessageTriggeredRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ConsoleMessageTriggeredRequest(f_))) - def portCompletedRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest] = field(_.getPortCompletedRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PortCompletedRequest(f_))) - def workerStateUpdatedRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest] = field(_.getWorkerStateUpdatedRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.WorkerStateUpdatedRequest(f_))) - def linkWorkersRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest] = field(_.getLinkWorkersRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.LinkWorkersRequest(f_))) - def addInputChannelRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest] = field(_.getAddInputChannelRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AddInputChannelRequest(f_))) - def addPartitioningRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest] = field(_.getAddPartitioningRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AddPartitioningRequest(f_))) - def assignPortRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest] = field(_.getAssignPortRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.AssignPortRequest(f_))) - def finalizeCheckpointRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest] = field(_.getFinalizeCheckpointRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.FinalizeCheckpointRequest(f_))) - def initializeExecutorRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest] = field(_.getInitializeExecutorRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.InitializeExecutorRequest(f_))) - def updateExecutorRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest] = field(_.getUpdateExecutorRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.UpdateExecutorRequest(f_))) - def emptyRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest] = field(_.getEmptyRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.EmptyRequest(f_))) - def prepareCheckpointRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest] = field(_.getPrepareCheckpointRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.PrepareCheckpointRequest(f_))) - def queryStatisticsRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest] = field(_.getQueryStatisticsRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.QueryStatisticsRequest(f_))) - def ping: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping] = field(_.getPing)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Ping(f_))) - def pong: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong] = field(_.getPong)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Pong(f_))) - def nested: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested] = field(_.getNested)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Nested(f_))) - def pass: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass] = field(_.getPass)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Pass(f_))) - def errorCommand: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand] = field(_.getErrorCommand)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.ErrorCommand(f_))) - def recursion: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion] = field(_.getRecursion)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Recursion(f_))) - def collect: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect] = field(_.getCollect)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Collect(f_))) - def generateNumber: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber] = field(_.getGenerateNumber)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.GenerateNumber(f_))) - def multiCall: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall] = field(_.getMultiCall)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.MultiCall(f_))) - def chain: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain] = field(_.getChain)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue.Chain(f_))) - def sealedValue: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue] = field(_.sealedValue)((c_, f_) => c_.copy(sealedValue = f_)) + implicit class ControlRequestMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage](_l) { + def propagateChannelMarkerRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest] = field(_.getPropagateChannelMarkerRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PropagateChannelMarkerRequest(f_))) + def takeGlobalCheckpointRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest] = field(_.getTakeGlobalCheckpointRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.TakeGlobalCheckpointRequest(f_))) + def debugCommandRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest] = field(_.getDebugCommandRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.DebugCommandRequest(f_))) + def evaluatePythonExpressionRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest] = field(_.getEvaluatePythonExpressionRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.EvaluatePythonExpressionRequest(f_))) + def modifyLogicRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest] = field(_.getModifyLogicRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ModifyLogicRequest(f_))) + def retryWorkflowRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest] = field(_.getRetryWorkflowRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.RetryWorkflowRequest(f_))) + def consoleMessageTriggeredRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest] = field(_.getConsoleMessageTriggeredRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ConsoleMessageTriggeredRequest(f_))) + def portCompletedRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest] = field(_.getPortCompletedRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PortCompletedRequest(f_))) + def workerStateUpdatedRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest] = field(_.getWorkerStateUpdatedRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.WorkerStateUpdatedRequest(f_))) + def linkWorkersRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest] = field(_.getLinkWorkersRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.LinkWorkersRequest(f_))) + def addInputChannelRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest] = field(_.getAddInputChannelRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AddInputChannelRequest(f_))) + def addPartitioningRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest] = field(_.getAddPartitioningRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AddPartitioningRequest(f_))) + def assignPortRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest] = field(_.getAssignPortRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.AssignPortRequest(f_))) + def finalizeCheckpointRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest] = field(_.getFinalizeCheckpointRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.FinalizeCheckpointRequest(f_))) + def initializeExecutorRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest] = field(_.getInitializeExecutorRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.InitializeExecutorRequest(f_))) + def updateExecutorRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest] = field(_.getUpdateExecutorRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.UpdateExecutorRequest(f_))) + def emptyRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest] = field(_.getEmptyRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.EmptyRequest(f_))) + def prepareCheckpointRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest] = field(_.getPrepareCheckpointRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.PrepareCheckpointRequest(f_))) + def queryStatisticsRequest: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest] = field(_.getQueryStatisticsRequest)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.QueryStatisticsRequest(f_))) + def ping: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Ping] = field(_.getPing)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Ping(f_))) + def pong: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Pong] = field(_.getPong)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Pong(f_))) + def nested: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Nested] = field(_.getNested)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Nested(f_))) + def pass: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Pass] = field(_.getPass)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Pass(f_))) + def errorCommand: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand] = field(_.getErrorCommand)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.ErrorCommand(f_))) + def recursion: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Recursion] = field(_.getRecursion)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Recursion(f_))) + def collect: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Collect] = field(_.getCollect)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Collect(f_))) + def generateNumber: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber] = field(_.getGenerateNumber)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.GenerateNumber(f_))) + def multiCall: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.MultiCall] = field(_.getMultiCall)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.MultiCall(f_))) + def chain: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Chain] = field(_.getChain)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue.Chain(f_))) + def sealedValue: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue] = field(_.sealedValue)((c_, f_) => c_.copy(sealedValue = f_)) } - final val PROPAGATECHANNELMARKERREQUEST_FIELD_NUMBER = 1 - final val TAKEGLOBALCHECKPOINTREQUEST_FIELD_NUMBER = 2 - final val DEBUGCOMMANDREQUEST_FIELD_NUMBER = 3 - final val EVALUATEPYTHONEXPRESSIONREQUEST_FIELD_NUMBER = 4 - final val MODIFYLOGICREQUEST_FIELD_NUMBER = 5 - final val RETRYWORKFLOWREQUEST_FIELD_NUMBER = 6 - final val CONSOLEMESSAGETRIGGEREDREQUEST_FIELD_NUMBER = 8 - final val PORTCOMPLETEDREQUEST_FIELD_NUMBER = 9 - final val WORKERSTATEUPDATEDREQUEST_FIELD_NUMBER = 10 - final val LINKWORKERSREQUEST_FIELD_NUMBER = 11 - final val ADDINPUTCHANNELREQUEST_FIELD_NUMBER = 50 - final val ADDPARTITIONINGREQUEST_FIELD_NUMBER = 51 - final val ASSIGNPORTREQUEST_FIELD_NUMBER = 52 - final val FINALIZECHECKPOINTREQUEST_FIELD_NUMBER = 53 - final val INITIALIZEEXECUTORREQUEST_FIELD_NUMBER = 54 - final val UPDATEEXECUTORREQUEST_FIELD_NUMBER = 55 - final val EMPTYREQUEST_FIELD_NUMBER = 56 - final val PREPARECHECKPOINTREQUEST_FIELD_NUMBER = 57 - final val QUERYSTATISTICSREQUEST_FIELD_NUMBER = 58 + final val PROPAGATE_CHANNEL_MARKER_REQUEST_FIELD_NUMBER = 1 + final val TAKE_GLOBAL_CHECKPOINT_REQUEST_FIELD_NUMBER = 2 + final val DEBUG_COMMAND_REQUEST_FIELD_NUMBER = 3 + final val EVALUATE_PYTHON_EXPRESSION_REQUEST_FIELD_NUMBER = 4 + final val MODIFY_LOGIC_REQUEST_FIELD_NUMBER = 5 + final val RETRY_WORKFLOW_REQUEST_FIELD_NUMBER = 6 + final val CONSOLE_MESSAGE_TRIGGERED_REQUEST_FIELD_NUMBER = 8 + final val PORT_COMPLETED_REQUEST_FIELD_NUMBER = 9 + final val WORKER_STATE_UPDATED_REQUEST_FIELD_NUMBER = 10 + final val LINK_WORKERS_REQUEST_FIELD_NUMBER = 11 + final val ADD_INPUT_CHANNEL_REQUEST_FIELD_NUMBER = 50 + final val ADD_PARTITIONING_REQUEST_FIELD_NUMBER = 51 + final val ASSIGN_PORT_REQUEST_FIELD_NUMBER = 52 + final val FINALIZE_CHECKPOINT_REQUEST_FIELD_NUMBER = 53 + final val INITIALIZE_EXECUTOR_REQUEST_FIELD_NUMBER = 54 + final val UPDATE_EXECUTOR_REQUEST_FIELD_NUMBER = 55 + final val EMPTY_REQUEST_FIELD_NUMBER = 56 + final val PREPARE_CHECKPOINT_REQUEST_FIELD_NUMBER = 57 + final val QUERY_STATISTICS_REQUEST_FIELD_NUMBER = 58 final val PING_FIELD_NUMBER = 100 final val PONG_FIELD_NUMBER = 101 final val NESTED_FIELD_NUMBER = 102 final val PASS_FIELD_NUMBER = 103 - final val ERRORCOMMAND_FIELD_NUMBER = 104 + final val ERROR_COMMAND_FIELD_NUMBER = 104 final val RECURSION_FIELD_NUMBER = 105 final val COLLECT_FIELD_NUMBER = 106 - final val GENERATENUMBER_FIELD_NUMBER = 107 - final val MULTICALL_FIELD_NUMBER = 108 + final val GENERATE_NUMBER_FIELD_NUMBER = 107 + final val MULTI_CALL_FIELD_NUMBER = 108 final val CHAIN_FIELD_NUMBER = 109 def of( - sealedValue: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.SealedValue - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage( + sealedValue: edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.SealedValue + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage = _root_.edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage( sealedValue ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ControlRequest]) @@ -1030,14 +1030,14 @@ object ControlRequestMessage extends scalapb.GeneratedMessageCompanion[edu.uci.i @SerialVersionUID(0L) final case class PropagateChannelMarkerRequest( - sourceOpToStartProp: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity], - id: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity, - markerType: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType, - scope: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity], - targetOps: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity], - markerCommand: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest, + sourceOpToStartProp: _root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity], + id: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity, + markerType: edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType, + scope: _root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity], + targetOps: _root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity], + markerCommand: edu.uci.ics.amber.engine.architecture.rpc.ControlRequest, markerMethodName: _root_.scala.Predef.String - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[PropagateChannelMarkerRequest] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[PropagateChannelMarkerRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1070,7 +1070,7 @@ final case class PropagateChannelMarkerRequest( } { - val __value = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest._typemapper_markerCommand.toBase(markerCommand) + val __value = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest._typemapper_markerCommand.toBase(markerCommand) if (__value.serializedSize != 0) { __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } @@ -1127,7 +1127,7 @@ final case class PropagateChannelMarkerRequest( __m.writeTo(_output__) }; { - val __v = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest._typemapper_markerCommand.toBase(markerCommand) + val __v = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest._typemapper_markerCommand.toBase(markerCommand) if (__v.serializedSize != 0) { _output__.writeTag(6, 2) _output__.writeUInt32NoTag(__v.serializedSize) @@ -1142,27 +1142,27 @@ final case class PropagateChannelMarkerRequest( }; } def clearSourceOpToStartProp = copy(sourceOpToStartProp = _root_.scala.Seq.empty) - def addSourceOpToStartProp(__vs: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity *): PropagateChannelMarkerRequest = addAllSourceOpToStartProp(__vs) - def addAllSourceOpToStartProp(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]): PropagateChannelMarkerRequest = copy(sourceOpToStartProp = sourceOpToStartProp ++ __vs) - def withSourceOpToStartProp(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]): PropagateChannelMarkerRequest = copy(sourceOpToStartProp = __v) - def withId(__v: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity): PropagateChannelMarkerRequest = copy(id = __v) - def withMarkerType(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType): PropagateChannelMarkerRequest = copy(markerType = __v) + def addSourceOpToStartProp(__vs: edu.uci.ics.amber.engine.common.PhysicalOpIdentity *): PropagateChannelMarkerRequest = addAllSourceOpToStartProp(__vs) + def addAllSourceOpToStartProp(__vs: Iterable[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]): PropagateChannelMarkerRequest = copy(sourceOpToStartProp = sourceOpToStartProp ++ __vs) + def withSourceOpToStartProp(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]): PropagateChannelMarkerRequest = copy(sourceOpToStartProp = __v) + def withId(__v: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity): PropagateChannelMarkerRequest = copy(id = __v) + def withMarkerType(__v: edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType): PropagateChannelMarkerRequest = copy(markerType = __v) def clearScope = copy(scope = _root_.scala.Seq.empty) - def addScope(__vs: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity *): PropagateChannelMarkerRequest = addAllScope(__vs) - def addAllScope(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]): PropagateChannelMarkerRequest = copy(scope = scope ++ __vs) - def withScope(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]): PropagateChannelMarkerRequest = copy(scope = __v) + def addScope(__vs: edu.uci.ics.amber.engine.common.PhysicalOpIdentity *): PropagateChannelMarkerRequest = addAllScope(__vs) + def addAllScope(__vs: Iterable[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]): PropagateChannelMarkerRequest = copy(scope = scope ++ __vs) + def withScope(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]): PropagateChannelMarkerRequest = copy(scope = __v) def clearTargetOps = copy(targetOps = _root_.scala.Seq.empty) - def addTargetOps(__vs: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity *): PropagateChannelMarkerRequest = addAllTargetOps(__vs) - def addAllTargetOps(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]): PropagateChannelMarkerRequest = copy(targetOps = targetOps ++ __vs) - def withTargetOps(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]): PropagateChannelMarkerRequest = copy(targetOps = __v) - def withMarkerCommand(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest): PropagateChannelMarkerRequest = copy(markerCommand = __v) + def addTargetOps(__vs: edu.uci.ics.amber.engine.common.PhysicalOpIdentity *): PropagateChannelMarkerRequest = addAllTargetOps(__vs) + def addAllTargetOps(__vs: Iterable[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]): PropagateChannelMarkerRequest = copy(targetOps = targetOps ++ __vs) + def withTargetOps(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]): PropagateChannelMarkerRequest = copy(targetOps = __v) + def withMarkerCommand(__v: edu.uci.ics.amber.engine.architecture.rpc.ControlRequest): PropagateChannelMarkerRequest = copy(markerCommand = __v) def withMarkerMethodName(__v: _root_.scala.Predef.String): PropagateChannelMarkerRequest = copy(markerMethodName = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => sourceOpToStartProp case 2 => { val __t = id - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance) __t else null } case 3 => { val __t = markerType.javaValueDescriptor @@ -1171,8 +1171,8 @@ final case class PropagateChannelMarkerRequest( case 4 => scope case 5 => targetOps case 6 => { - val __t = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest._typemapper_markerCommand.toBase(markerCommand) - if (__t != edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.defaultInstance) __t else null + val __t = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest._typemapper_markerCommand.toBase(markerCommand) + if (__t != edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.defaultInstance) __t else null } case 7 => { val __t = markerMethodName @@ -1188,24 +1188,24 @@ final case class PropagateChannelMarkerRequest( case 3 => _root_.scalapb.descriptors.PEnum(markerType.scalaValueDescriptor) case 4 => _root_.scalapb.descriptors.PRepeated(scope.iterator.map(_.toPMessage).toVector) case 5 => _root_.scalapb.descriptors.PRepeated(targetOps.iterator.map(_.toPMessage).toVector) - case 6 => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest._typemapper_markerCommand.toBase(markerCommand).toPMessage + case 6 => edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest._typemapper_markerCommand.toBase(markerCommand).toPMessage case 7 => _root_.scalapb.descriptors.PString(markerMethodName) } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest.type = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest]) } -object PropagateChannelMarkerRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest = { - val __sourceOpToStartProp: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] - var __id: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] = _root_.scala.None - var __markerType: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.REQUIRE_ALIGNMENT - val __scope: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] - val __targetOps: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] - var __markerCommand: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage] = _root_.scala.None +object PropagateChannelMarkerRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest = { + val __sourceOpToStartProp: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.PhysicalOpIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.PhysicalOpIdentity] + var __id: _root_.scala.Option[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] = _root_.scala.None + var __markerType: edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.REQUIRE_ALIGNMENT + val __scope: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.PhysicalOpIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.PhysicalOpIdentity] + val __targetOps: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.PhysicalOpIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.PhysicalOpIdentity] + var __markerCommand: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage] = _root_.scala.None var __markerMethodName: _root_.scala.Predef.String = "" var _done__ = false while (!_done__) { @@ -1213,101 +1213,101 @@ object PropagateChannelMarkerRequest extends scalapb.GeneratedMessageCompanion[e _tag__ match { case 0 => _done__ = true case 10 => - __sourceOpToStartProp += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity](_input__) + __sourceOpToStartProp += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PhysicalOpIdentity](_input__) case 18 => - __id = _root_.scala.Some(__id.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __id = _root_.scala.Some(__id.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 24 => - __markerType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.fromValue(_input__.readEnum()) + __markerType = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.fromValue(_input__.readEnum()) case 34 => - __scope += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity](_input__) + __scope += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PhysicalOpIdentity](_input__) case 42 => - __targetOps += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity](_input__) + __targetOps += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PhysicalOpIdentity](_input__) case 50 => - __markerCommand = _root_.scala.Some(__markerCommand.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __markerCommand = _root_.scala.Some(__markerCommand.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 58 => __markerMethodName = _input__.readStringRequireUtf8() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest( + edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest( sourceOpToStartProp = __sourceOpToStartProp.result(), - id = __id.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance), + id = __id.getOrElse(edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance), markerType = __markerType, scope = __scope.result(), targetOps = __targetOps.result(), - markerCommand = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest._typemapper_markerCommand.toCustom(__markerCommand.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.defaultInstance)), + markerCommand = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest._typemapper_markerCommand.toCustom(__markerCommand.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.defaultInstance)), markerMethodName = __markerMethodName ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest( - sourceOpToStartProp = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]]).getOrElse(_root_.scala.Seq.empty), - id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance), - markerType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.REQUIRE_ALIGNMENT.scalaValueDescriptor).number), - scope = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]]).getOrElse(_root_.scala.Seq.empty), - targetOps = __fieldsMap.get(scalaDescriptor.findFieldByNumber(5).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]]).getOrElse(_root_.scala.Seq.empty), - markerCommand = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest._typemapper_markerCommand.toCustom(__fieldsMap.get(scalaDescriptor.findFieldByNumber(6).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.defaultInstance)), + edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest( + sourceOpToStartProp = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]]).getOrElse(_root_.scala.Seq.empty), + id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance), + markerType = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.REQUIRE_ALIGNMENT.scalaValueDescriptor).number), + scope = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]]).getOrElse(_root_.scala.Seq.empty), + targetOps = __fieldsMap.get(scalaDescriptor.findFieldByNumber(5).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]]).getOrElse(_root_.scala.Seq.empty), + markerCommand = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest._typemapper_markerCommand.toCustom(__fieldsMap.get(scalaDescriptor.findFieldByNumber(6).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.defaultInstance)), markerMethodName = __fieldsMap.get(scalaDescriptor.findFieldByNumber(7).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(5) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(5) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(5) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(5) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity - case 2 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity - case 4 => __out = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity - case 5 => __out = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity - case 6 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage + case 1 => __out = edu.uci.ics.amber.engine.common.PhysicalOpIdentity + case 2 => __out = edu.uci.ics.amber.engine.common.ChannelMarkerIdentity + case 4 => __out = edu.uci.ics.amber.engine.common.PhysicalOpIdentity + case 5 => __out = edu.uci.ics.amber.engine.common.PhysicalOpIdentity + case 6 => __out = edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = { (__fieldNumber: @_root_.scala.unchecked) match { - case 3 => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType + case 3 => edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType } } - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest( sourceOpToStartProp = _root_.scala.Seq.empty, - id = edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance, - markerType = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType.REQUIRE_ALIGNMENT, + id = edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance, + markerType = edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.REQUIRE_ALIGNMENT, scope = _root_.scala.Seq.empty, targetOps = _root_.scala.Seq.empty, - markerCommand = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest._typemapper_markerCommand.toCustom(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage.defaultInstance), + markerCommand = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest._typemapper_markerCommand.toCustom(edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage.defaultInstance), markerMethodName = "" ) - implicit class PropagateChannelMarkerRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest](_l) { - def sourceOpToStartProp: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]] = field(_.sourceOpToStartProp)((c_, f_) => c_.copy(sourceOpToStartProp = f_)) - def id: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] = field(_.id)((c_, f_) => c_.copy(id = f_)) - def markerType: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType] = field(_.markerType)((c_, f_) => c_.copy(markerType = f_)) - def scope: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]] = field(_.scope)((c_, f_) => c_.copy(scope = f_)) - def targetOps: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]] = field(_.targetOps)((c_, f_) => c_.copy(targetOps = f_)) - def markerCommand: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest] = field(_.markerCommand)((c_, f_) => c_.copy(markerCommand = f_)) + implicit class PropagateChannelMarkerRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest](_l) { + def sourceOpToStartProp: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]] = field(_.sourceOpToStartProp)((c_, f_) => c_.copy(sourceOpToStartProp = f_)) + def id: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] = field(_.id)((c_, f_) => c_.copy(id = f_)) + def markerType: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType] = field(_.markerType)((c_, f_) => c_.copy(markerType = f_)) + def scope: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]] = field(_.scope)((c_, f_) => c_.copy(scope = f_)) + def targetOps: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]] = field(_.targetOps)((c_, f_) => c_.copy(targetOps = f_)) + def markerCommand: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlRequest] = field(_.markerCommand)((c_, f_) => c_.copy(markerCommand = f_)) def markerMethodName: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.markerMethodName)((c_, f_) => c_.copy(markerMethodName = f_)) } - final val SOURCEOPTOSTARTPROP_FIELD_NUMBER = 1 + final val SOURCE_OP_TO_START_PROP_FIELD_NUMBER = 1 final val ID_FIELD_NUMBER = 2 - final val MARKERTYPE_FIELD_NUMBER = 3 + final val MARKER_TYPE_FIELD_NUMBER = 3 final val SCOPE_FIELD_NUMBER = 4 - final val TARGETOPS_FIELD_NUMBER = 5 - final val MARKERCOMMAND_FIELD_NUMBER = 6 - final val MARKERMETHODNAME_FIELD_NUMBER = 7 + final val TARGET_OPS_FIELD_NUMBER = 5 + final val MARKER_COMMAND_FIELD_NUMBER = 6 + final val MARKER_METHOD_NAME_FIELD_NUMBER = 7 @transient - private[controlcommands] val _typemapper_markerCommand: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest]] + private[rpc] val _typemapper_markerCommand: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage, edu.uci.ics.amber.engine.architecture.rpc.ControlRequest] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ControlRequestMessage, edu.uci.ics.amber.engine.architecture.rpc.ControlRequest]] def of( - sourceOpToStartProp: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity], - id: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity, - markerType: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerType, - scope: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity], - targetOps: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity], - markerCommand: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest, + sourceOpToStartProp: _root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity], + id: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity, + markerType: edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType, + scope: _root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity], + targetOps: _root_.scala.Seq[edu.uci.ics.amber.engine.common.PhysicalOpIdentity], + markerCommand: edu.uci.ics.amber.engine.architecture.rpc.ControlRequest, markerMethodName: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest( sourceOpToStartProp, id, markerType, @@ -1322,9 +1322,9 @@ object PropagateChannelMarkerRequest extends scalapb.GeneratedMessageCompanion[e @SerialVersionUID(0L) final case class TakeGlobalCheckpointRequest( estimationOnly: _root_.scala.Boolean, - checkpointId: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity, + checkpointId: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity, destination: _root_.scala.Predef.String - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[TakeGlobalCheckpointRequest] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[TakeGlobalCheckpointRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1384,7 +1384,7 @@ final case class TakeGlobalCheckpointRequest( }; } def withEstimationOnly(__v: _root_.scala.Boolean): TakeGlobalCheckpointRequest = copy(estimationOnly = __v) - def withCheckpointId(__v: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity): TakeGlobalCheckpointRequest = copy(checkpointId = __v) + def withCheckpointId(__v: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity): TakeGlobalCheckpointRequest = copy(checkpointId = __v) def withDestination(__v: _root_.scala.Predef.String): TakeGlobalCheckpointRequest = copy(destination = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { @@ -1394,7 +1394,7 @@ final case class TakeGlobalCheckpointRequest( } case 2 => { val __t = checkpointId - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance) __t else null } case 3 => { val __t = destination @@ -1411,15 +1411,15 @@ final case class TakeGlobalCheckpointRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest.type = edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest]) } -object TakeGlobalCheckpointRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest = { +object TakeGlobalCheckpointRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest = { var __estimationOnly: _root_.scala.Boolean = false - var __checkpointId: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] = _root_.scala.None + var __checkpointId: _root_.scala.Option[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] = _root_.scala.None var __destination: _root_.scala.Predef.String = "" var _done__ = false while (!_done__) { @@ -1429,57 +1429,57 @@ object TakeGlobalCheckpointRequest extends scalapb.GeneratedMessageCompanion[edu case 8 => __estimationOnly = _input__.readBool() case 18 => - __checkpointId = _root_.scala.Some(__checkpointId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __checkpointId = _root_.scala.Some(__checkpointId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 26 => __destination = _input__.readStringRequireUtf8() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest( + edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest( estimationOnly = __estimationOnly, - checkpointId = __checkpointId.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance), + checkpointId = __checkpointId.getOrElse(edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance), destination = __destination ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest( + edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest( estimationOnly = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Boolean]).getOrElse(false), - checkpointId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance), + checkpointId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance), destination = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(6) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(6) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(6) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(6) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity + case 2 => __out = edu.uci.ics.amber.engine.common.ChannelMarkerIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest( estimationOnly = false, - checkpointId = edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance, + checkpointId = edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance, destination = "" ) - implicit class TakeGlobalCheckpointRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest](_l) { + implicit class TakeGlobalCheckpointRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest](_l) { def estimationOnly: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Boolean] = field(_.estimationOnly)((c_, f_) => c_.copy(estimationOnly = f_)) - def checkpointId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] = field(_.checkpointId)((c_, f_) => c_.copy(checkpointId = f_)) + def checkpointId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] = field(_.checkpointId)((c_, f_) => c_.copy(checkpointId = f_)) def destination: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.destination)((c_, f_) => c_.copy(destination = f_)) } - final val ESTIMATIONONLY_FIELD_NUMBER = 1 - final val CHECKPOINTID_FIELD_NUMBER = 2 + final val ESTIMATION_ONLY_FIELD_NUMBER = 1 + final val CHECKPOINT_ID_FIELD_NUMBER = 2 final val DESTINATION_FIELD_NUMBER = 3 def of( estimationOnly: _root_.scala.Boolean, - checkpointId: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity, + checkpointId: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity, destination: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest( estimationOnly, checkpointId, destination @@ -1491,7 +1491,7 @@ object TakeGlobalCheckpointRequest extends scalapb.GeneratedMessageCompanion[edu final case class DebugCommandRequest( workerId: _root_.scala.Predef.String, cmd: _root_.scala.Predef.String - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[DebugCommandRequest] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[DebugCommandRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1557,13 +1557,13 @@ final case class DebugCommandRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest.type = edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest]) } -object DebugCommandRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest = { +object DebugCommandRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest = { var __workerId: _root_.scala.Predef.String = "" var __cmd: _root_.scala.Predef.String = "" var _done__ = false @@ -1578,39 +1578,39 @@ object DebugCommandRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest( + edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest( workerId = __workerId, cmd = __cmd ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest( + edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest( workerId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), cmd = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(8) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(8) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(8) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(8) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest( workerId = "", cmd = "" ) - implicit class DebugCommandRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest](_l) { + implicit class DebugCommandRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest](_l) { def workerId: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.workerId)((c_, f_) => c_.copy(workerId = f_)) def cmd: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.cmd)((c_, f_) => c_.copy(cmd = f_)) } - final val WORKERID_FIELD_NUMBER = 1 + final val WORKER_ID_FIELD_NUMBER = 1 final val CMD_FIELD_NUMBER = 2 def of( workerId: _root_.scala.Predef.String, cmd: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest( workerId, cmd ) @@ -1621,7 +1621,7 @@ object DebugCommandRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics final case class EvaluatePythonExpressionRequest( expression: _root_.scala.Predef.String, operatorId: _root_.scala.Predef.String - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[EvaluatePythonExpressionRequest] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[EvaluatePythonExpressionRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1687,13 +1687,13 @@ final case class EvaluatePythonExpressionRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest.type = edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest]) } -object EvaluatePythonExpressionRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest = { +object EvaluatePythonExpressionRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest = { var __expression: _root_.scala.Predef.String = "" var __operatorId: _root_.scala.Predef.String = "" var _done__ = false @@ -1708,39 +1708,39 @@ object EvaluatePythonExpressionRequest extends scalapb.GeneratedMessageCompanion case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest( + edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest( expression = __expression, operatorId = __operatorId ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest( + edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest( expression = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), operatorId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(9) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(9) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(9) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(9) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest( expression = "", operatorId = "" ) - implicit class EvaluatePythonExpressionRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest](_l) { + implicit class EvaluatePythonExpressionRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest](_l) { def expression: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.expression)((c_, f_) => c_.copy(expression = f_)) def operatorId: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.operatorId)((c_, f_) => c_.copy(operatorId = f_)) } final val EXPRESSION_FIELD_NUMBER = 1 - final val OPERATORID_FIELD_NUMBER = 2 + final val OPERATOR_ID_FIELD_NUMBER = 2 def of( expression: _root_.scala.Predef.String, operatorId: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest( expression, operatorId ) @@ -1749,8 +1749,8 @@ object EvaluatePythonExpressionRequest extends scalapb.GeneratedMessageCompanion @SerialVersionUID(0L) final case class ModifyLogicRequest( - updateRequest: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[ModifyLogicRequest] { + updateRequest: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest] + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[ModifyLogicRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1779,9 +1779,9 @@ final case class ModifyLogicRequest( }; } def clearUpdateRequest = copy(updateRequest = _root_.scala.Seq.empty) - def addUpdateRequest(__vs: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest *): ModifyLogicRequest = addAllUpdateRequest(__vs) - def addAllUpdateRequest(__vs: Iterable[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest]): ModifyLogicRequest = copy(updateRequest = updateRequest ++ __vs) - def withUpdateRequest(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest]): ModifyLogicRequest = copy(updateRequest = __v) + def addUpdateRequest(__vs: edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest *): ModifyLogicRequest = addAllUpdateRequest(__vs) + def addAllUpdateRequest(__vs: Iterable[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest]): ModifyLogicRequest = copy(updateRequest = updateRequest ++ __vs) + def withUpdateRequest(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest]): ModifyLogicRequest = copy(updateRequest = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => updateRequest @@ -1794,57 +1794,57 @@ final case class ModifyLogicRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest.type = edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest]) } -object ModifyLogicRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest = { - val __updateRequest: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest] +object ModifyLogicRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest = { + val __updateRequest: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __updateRequest += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest](_input__) + __updateRequest += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest( + edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest( updateRequest = __updateRequest.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest( - updateRequest = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest]]).getOrElse(_root_.scala.Seq.empty) + edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest( + updateRequest = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(10) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(10) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(10) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(10) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest + case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest( updateRequest = _root_.scala.Seq.empty ) - implicit class ModifyLogicRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest](_l) { - def updateRequest: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest]] = field(_.updateRequest)((c_, f_) => c_.copy(updateRequest = f_)) + implicit class ModifyLogicRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest](_l) { + def updateRequest: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest]] = field(_.updateRequest)((c_, f_) => c_.copy(updateRequest = f_)) } - final val UPDATEREQUEST_FIELD_NUMBER = 1 + final val UPDATE_REQUEST_FIELD_NUMBER = 1 def of( - updateRequest: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest( + updateRequest: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest] + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest( updateRequest ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest]) @@ -1852,8 +1852,8 @@ object ModifyLogicRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics. @SerialVersionUID(0L) final case class RetryWorkflowRequest( - workers: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[RetryWorkflowRequest] { + workers: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[RetryWorkflowRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1882,9 +1882,9 @@ final case class RetryWorkflowRequest( }; } def clearWorkers = copy(workers = _root_.scala.Seq.empty) - def addWorkers(__vs: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity *): RetryWorkflowRequest = addAllWorkers(__vs) - def addAllWorkers(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]): RetryWorkflowRequest = copy(workers = workers ++ __vs) - def withWorkers(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]): RetryWorkflowRequest = copy(workers = __v) + def addWorkers(__vs: edu.uci.ics.amber.engine.common.ActorVirtualIdentity *): RetryWorkflowRequest = addAllWorkers(__vs) + def addAllWorkers(__vs: Iterable[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]): RetryWorkflowRequest = copy(workers = workers ++ __vs) + def withWorkers(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]): RetryWorkflowRequest = copy(workers = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => workers @@ -1897,57 +1897,57 @@ final case class RetryWorkflowRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest.type = edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest]) } -object RetryWorkflowRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest = { - val __workers: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] +object RetryWorkflowRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest = { + val __workers: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __workers += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__) + __workers += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest( + edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest( workers = __workers.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest( - workers = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]]).getOrElse(_root_.scala.Seq.empty) + edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest( + workers = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(11) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(11) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(11) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(11) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest( workers = _root_.scala.Seq.empty ) - implicit class RetryWorkflowRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest](_l) { - def workers: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]] = field(_.workers)((c_, f_) => c_.copy(workers = f_)) + implicit class RetryWorkflowRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest](_l) { + def workers: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]] = field(_.workers)((c_, f_) => c_.copy(workers = f_)) } final val WORKERS_FIELD_NUMBER = 1 def of( - workers: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest( + workers: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest( workers ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest]) @@ -1955,8 +1955,8 @@ object RetryWorkflowRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ic @SerialVersionUID(0L) final case class ConsoleMessageTriggeredRequest( - consoleMessage: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[ConsoleMessageTriggeredRequest] { + consoleMessage: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[ConsoleMessageTriggeredRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1989,12 +1989,12 @@ final case class ConsoleMessageTriggeredRequest( } }; } - def withConsoleMessage(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage): ConsoleMessageTriggeredRequest = copy(consoleMessage = __v) + def withConsoleMessage(__v: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage): ConsoleMessageTriggeredRequest = copy(consoleMessage = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = consoleMessage - if (__t != edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage.defaultInstance) __t else null } } } @@ -2005,57 +2005,57 @@ final case class ConsoleMessageTriggeredRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest.type = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest]) } -object ConsoleMessageTriggeredRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest = { - var __consoleMessage: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage] = _root_.scala.None +object ConsoleMessageTriggeredRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest = { + var __consoleMessage: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __consoleMessage = _root_.scala.Some(__consoleMessage.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __consoleMessage = _root_.scala.Some(__consoleMessage.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest( - consoleMessage = __consoleMessage.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage.defaultInstance) + edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest( + consoleMessage = __consoleMessage.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage.defaultInstance) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest( - consoleMessage = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage.defaultInstance) + edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest( + consoleMessage = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage.defaultInstance) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(13) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(13) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(13) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(13) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage + case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest( - consoleMessage = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage.defaultInstance + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest( + consoleMessage = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage.defaultInstance ) - implicit class ConsoleMessageTriggeredRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest](_l) { - def consoleMessage: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage] = field(_.consoleMessage)((c_, f_) => c_.copy(consoleMessage = f_)) + implicit class ConsoleMessageTriggeredRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest](_l) { + def consoleMessage: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage] = field(_.consoleMessage)((c_, f_) => c_.copy(consoleMessage = f_)) } - final val CONSOLEMESSAGE_FIELD_NUMBER = 1 + final val CONSOLE_MESSAGE_FIELD_NUMBER = 1 def of( - consoleMessage: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest( + consoleMessage: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest( consoleMessage ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest]) @@ -2063,9 +2063,9 @@ object ConsoleMessageTriggeredRequest extends scalapb.GeneratedMessageCompanion[ @SerialVersionUID(0L) final case class PortCompletedRequest( - portId: edu.uci.ics.amber.engine.common.workflow.PortIdentity, + portId: edu.uci.ics.amber.engine.common.PortIdentity, input: _root_.scala.Boolean - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[PortCompletedRequest] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[PortCompletedRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -2111,13 +2111,13 @@ final case class PortCompletedRequest( } }; } - def withPortId(__v: edu.uci.ics.amber.engine.common.workflow.PortIdentity): PortCompletedRequest = copy(portId = __v) + def withPortId(__v: edu.uci.ics.amber.engine.common.PortIdentity): PortCompletedRequest = copy(portId = __v) def withInput(__v: _root_.scala.Boolean): PortCompletedRequest = copy(input = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = portId - if (__t != edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance) __t else null } case 2 => { val __t = input @@ -2133,14 +2133,14 @@ final case class PortCompletedRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest.type = edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest]) } -object PortCompletedRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest = { - var __portId: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflow.PortIdentity] = _root_.scala.None +object PortCompletedRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest = { + var __portId: _root_.scala.Option[edu.uci.ics.amber.engine.common.PortIdentity] = _root_.scala.None var __input: _root_.scala.Boolean = false var _done__ = false while (!_done__) { @@ -2148,51 +2148,51 @@ object PortCompletedRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ic _tag__ match { case 0 => _done__ = true case 10 => - __portId = _root_.scala.Some(__portId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflow.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __portId = _root_.scala.Some(__portId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 16 => __input = _input__.readBool() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest( - portId = __portId.getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest( + portId = __portId.getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance), input = __input ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest( - portId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.workflow.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest( + portId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance), input = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Boolean]).getOrElse(false) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(14) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(14) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(14) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(14) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.workflow.PortIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.PortIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest( - portId = edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance, + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest( + portId = edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance, input = false ) - implicit class PortCompletedRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest](_l) { - def portId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PortIdentity] = field(_.portId)((c_, f_) => c_.copy(portId = f_)) + implicit class PortCompletedRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest](_l) { + def portId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PortIdentity] = field(_.portId)((c_, f_) => c_.copy(portId = f_)) def input: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Boolean] = field(_.input)((c_, f_) => c_.copy(input = f_)) } - final val PORTID_FIELD_NUMBER = 1 + final val PORT_ID_FIELD_NUMBER = 1 final val INPUT_FIELD_NUMBER = 2 def of( - portId: edu.uci.ics.amber.engine.common.workflow.PortIdentity, + portId: edu.uci.ics.amber.engine.common.PortIdentity, input: _root_.scala.Boolean - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest( portId, input ) @@ -2201,8 +2201,8 @@ object PortCompletedRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ic @SerialVersionUID(0L) final case class WorkerStateUpdatedRequest( - state: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[WorkerStateUpdatedRequest] { + state: edu.uci.ics.amber.engine.architecture.worker.WorkerState + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[WorkerStateUpdatedRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -2233,7 +2233,7 @@ final case class WorkerStateUpdatedRequest( } }; } - def withState(__v: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState): WorkerStateUpdatedRequest = copy(state = __v) + def withState(__v: edu.uci.ics.amber.engine.architecture.worker.WorkerState): WorkerStateUpdatedRequest = copy(state = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -2249,55 +2249,55 @@ final case class WorkerStateUpdatedRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest.type = edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest]) } -object WorkerStateUpdatedRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest = { - var __state: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.UNINITIALIZED +object WorkerStateUpdatedRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest = { + var __state: edu.uci.ics.amber.engine.architecture.worker.WorkerState = edu.uci.ics.amber.engine.architecture.worker.WorkerState.UNINITIALIZED var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 8 => - __state = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.fromValue(_input__.readEnum()) + __state = edu.uci.ics.amber.engine.architecture.worker.WorkerState.fromValue(_input__.readEnum()) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest( + edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest( state = __state ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest( - state = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.UNINITIALIZED.scalaValueDescriptor).number) + edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest( + state = edu.uci.ics.amber.engine.architecture.worker.WorkerState.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.worker.WorkerState.UNINITIALIZED.scalaValueDescriptor).number) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(15) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(15) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(15) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(15) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = { (__fieldNumber: @_root_.scala.unchecked) match { - case 1 => edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState + case 1 => edu.uci.ics.amber.engine.architecture.worker.WorkerState } } - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest( - state = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.UNINITIALIZED + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest( + state = edu.uci.ics.amber.engine.architecture.worker.WorkerState.UNINITIALIZED ) - implicit class WorkerStateUpdatedRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest](_l) { - def state: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState] = field(_.state)((c_, f_) => c_.copy(state = f_)) + implicit class WorkerStateUpdatedRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest](_l) { + def state: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.WorkerState] = field(_.state)((c_, f_) => c_.copy(state = f_)) } final val STATE_FIELD_NUMBER = 1 def of( - state: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest( + state: edu.uci.ics.amber.engine.architecture.worker.WorkerState + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest( state ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest]) @@ -2305,8 +2305,8 @@ object WorkerStateUpdatedRequest extends scalapb.GeneratedMessageCompanion[edu.u @SerialVersionUID(0L) final case class LinkWorkersRequest( - link: edu.uci.ics.amber.engine.common.workflow.PhysicalLink - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[LinkWorkersRequest] { + link: edu.uci.ics.amber.engine.common.PhysicalLink + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[LinkWorkersRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -2339,12 +2339,12 @@ final case class LinkWorkersRequest( } }; } - def withLink(__v: edu.uci.ics.amber.engine.common.workflow.PhysicalLink): LinkWorkersRequest = copy(link = __v) + def withLink(__v: edu.uci.ics.amber.engine.common.PhysicalLink): LinkWorkersRequest = copy(link = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = link - if (__t != edu.uci.ics.amber.engine.common.workflow.PhysicalLink.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PhysicalLink.defaultInstance) __t else null } } } @@ -2355,57 +2355,57 @@ final case class LinkWorkersRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest.type = edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest]) } -object LinkWorkersRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest = { - var __link: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflow.PhysicalLink] = _root_.scala.None +object LinkWorkersRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest = { + var __link: _root_.scala.Option[edu.uci.ics.amber.engine.common.PhysicalLink] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __link = _root_.scala.Some(__link.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflow.PhysicalLink](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __link = _root_.scala.Some(__link.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PhysicalLink](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest( - link = __link.getOrElse(edu.uci.ics.amber.engine.common.workflow.PhysicalLink.defaultInstance) + edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest( + link = __link.getOrElse(edu.uci.ics.amber.engine.common.PhysicalLink.defaultInstance) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest( - link = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.workflow.PhysicalLink]).getOrElse(edu.uci.ics.amber.engine.common.workflow.PhysicalLink.defaultInstance) + edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest( + link = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.PhysicalLink]).getOrElse(edu.uci.ics.amber.engine.common.PhysicalLink.defaultInstance) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(16) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(16) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(16) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(16) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.workflow.PhysicalLink + case 1 => __out = edu.uci.ics.amber.engine.common.PhysicalLink } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest( - link = edu.uci.ics.amber.engine.common.workflow.PhysicalLink.defaultInstance + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest( + link = edu.uci.ics.amber.engine.common.PhysicalLink.defaultInstance ) - implicit class LinkWorkersRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest](_l) { - def link: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PhysicalLink] = field(_.link)((c_, f_) => c_.copy(link = f_)) + implicit class LinkWorkersRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest](_l) { + def link: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PhysicalLink] = field(_.link)((c_, f_) => c_.copy(link = f_)) } final val LINK_FIELD_NUMBER = 1 def of( - link: edu.uci.ics.amber.engine.common.workflow.PhysicalLink - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest( + link: edu.uci.ics.amber.engine.common.PhysicalLink + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest( link ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest]) @@ -2415,9 +2415,9 @@ object LinkWorkersRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics. */ @SerialVersionUID(0L) final case class AddInputChannelRequest( - channelId: edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity, - portId: edu.uci.ics.amber.engine.common.workflow.PortIdentity - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[AddInputChannelRequest] { + channelId: edu.uci.ics.amber.engine.common.ChannelIdentity, + portId: edu.uci.ics.amber.engine.common.PortIdentity + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[AddInputChannelRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -2465,17 +2465,17 @@ final case class AddInputChannelRequest( } }; } - def withChannelId(__v: edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity): AddInputChannelRequest = copy(channelId = __v) - def withPortId(__v: edu.uci.ics.amber.engine.common.workflow.PortIdentity): AddInputChannelRequest = copy(portId = __v) + def withChannelId(__v: edu.uci.ics.amber.engine.common.ChannelIdentity): AddInputChannelRequest = copy(channelId = __v) + def withPortId(__v: edu.uci.ics.amber.engine.common.PortIdentity): AddInputChannelRequest = copy(portId = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = channelId - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ChannelIdentity.defaultInstance) __t else null } case 2 => { val __t = portId - if (__t != edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance) __t else null } } } @@ -2487,67 +2487,67 @@ final case class AddInputChannelRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest.type = edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest]) } -object AddInputChannelRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest = { - var __channelId: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] = _root_.scala.None - var __portId: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflow.PortIdentity] = _root_.scala.None +object AddInputChannelRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest = { + var __channelId: _root_.scala.Option[edu.uci.ics.amber.engine.common.ChannelIdentity] = _root_.scala.None + var __portId: _root_.scala.Option[edu.uci.ics.amber.engine.common.PortIdentity] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __channelId = _root_.scala.Some(__channelId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __channelId = _root_.scala.Some(__channelId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ChannelIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => - __portId = _root_.scala.Some(__portId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflow.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __portId = _root_.scala.Some(__portId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest( - channelId = __channelId.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity.defaultInstance), - portId = __portId.getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance) + edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest( + channelId = __channelId.getOrElse(edu.uci.ics.amber.engine.common.ChannelIdentity.defaultInstance), + portId = __portId.getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest( - channelId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity.defaultInstance), - portId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.workflow.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance) + edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest( + channelId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.ChannelIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ChannelIdentity.defaultInstance), + portId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(27) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(27) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(27) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(27) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity - case 2 => __out = edu.uci.ics.amber.engine.common.workflow.PortIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.ChannelIdentity + case 2 => __out = edu.uci.ics.amber.engine.common.PortIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest( - channelId = edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity.defaultInstance, - portId = edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest( + channelId = edu.uci.ics.amber.engine.common.ChannelIdentity.defaultInstance, + portId = edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance ) - implicit class AddInputChannelRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest](_l) { - def channelId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] = field(_.channelId)((c_, f_) => c_.copy(channelId = f_)) - def portId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PortIdentity] = field(_.portId)((c_, f_) => c_.copy(portId = f_)) + implicit class AddInputChannelRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest](_l) { + def channelId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ChannelIdentity] = field(_.channelId)((c_, f_) => c_.copy(channelId = f_)) + def portId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PortIdentity] = field(_.portId)((c_, f_) => c_.copy(portId = f_)) } - final val CHANNELID_FIELD_NUMBER = 1 - final val PORTID_FIELD_NUMBER = 2 + final val CHANNEL_ID_FIELD_NUMBER = 1 + final val PORT_ID_FIELD_NUMBER = 2 def of( - channelId: edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity, - portId: edu.uci.ics.amber.engine.common.workflow.PortIdentity - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest( + channelId: edu.uci.ics.amber.engine.common.ChannelIdentity, + portId: edu.uci.ics.amber.engine.common.PortIdentity + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest( channelId, portId ) @@ -2556,9 +2556,9 @@ object AddInputChannelRequest extends scalapb.GeneratedMessageCompanion[edu.uci. @SerialVersionUID(0L) final case class AddPartitioningRequest( - tag: edu.uci.ics.amber.engine.common.workflow.PhysicalLink, - partitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[AddPartitioningRequest] { + tag: edu.uci.ics.amber.engine.common.PhysicalLink, + partitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[AddPartitioningRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -2572,7 +2572,7 @@ final case class AddPartitioningRequest( }; { - val __value = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest._typemapper_partitioning.toBase(partitioning) + val __value = edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest._typemapper_partitioning.toBase(partitioning) if (__value.serializedSize != 0) { __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } @@ -2598,7 +2598,7 @@ final case class AddPartitioningRequest( } }; { - val __v = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest._typemapper_partitioning.toBase(partitioning) + val __v = edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest._typemapper_partitioning.toBase(partitioning) if (__v.serializedSize != 0) { _output__.writeTag(2, 2) _output__.writeUInt32NoTag(__v.serializedSize) @@ -2606,17 +2606,17 @@ final case class AddPartitioningRequest( } }; } - def withTag(__v: edu.uci.ics.amber.engine.common.workflow.PhysicalLink): AddPartitioningRequest = copy(tag = __v) - def withPartitioning(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning): AddPartitioningRequest = copy(partitioning = __v) + def withTag(__v: edu.uci.ics.amber.engine.common.PhysicalLink): AddPartitioningRequest = copy(tag = __v) + def withPartitioning(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning): AddPartitioningRequest = copy(partitioning = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = tag - if (__t != edu.uci.ics.amber.engine.common.workflow.PhysicalLink.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PhysicalLink.defaultInstance) __t else null } case 2 => { - val __t = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest._typemapper_partitioning.toBase(partitioning) - if (__t != edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.defaultInstance) __t else null + val __t = edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest._typemapper_partitioning.toBase(partitioning) + if (__t != edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.defaultInstance) __t else null } } } @@ -2624,73 +2624,73 @@ final case class AddPartitioningRequest( _root_.scala.Predef.require(__field.containingMessage eq companion.scalaDescriptor) (__field.number: @_root_.scala.unchecked) match { case 1 => tag.toPMessage - case 2 => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest._typemapper_partitioning.toBase(partitioning).toPMessage + case 2 => edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest._typemapper_partitioning.toBase(partitioning).toPMessage } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest.type = edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest]) } -object AddPartitioningRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest = { - var __tag: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflow.PhysicalLink] = _root_.scala.None - var __partitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage] = _root_.scala.None +object AddPartitioningRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest = { + var __tag: _root_.scala.Option[edu.uci.ics.amber.engine.common.PhysicalLink] = _root_.scala.None + var __partitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __tag = _root_.scala.Some(__tag.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflow.PhysicalLink](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __tag = _root_.scala.Some(__tag.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PhysicalLink](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => - __partitioning = _root_.scala.Some(__partitioning.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __partitioning = _root_.scala.Some(__partitioning.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest( - tag = __tag.getOrElse(edu.uci.ics.amber.engine.common.workflow.PhysicalLink.defaultInstance), - partitioning = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest._typemapper_partitioning.toCustom(__partitioning.getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.defaultInstance)) + edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest( + tag = __tag.getOrElse(edu.uci.ics.amber.engine.common.PhysicalLink.defaultInstance), + partitioning = edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest._typemapper_partitioning.toCustom(__partitioning.getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.defaultInstance)) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest( - tag = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.workflow.PhysicalLink]).getOrElse(edu.uci.ics.amber.engine.common.workflow.PhysicalLink.defaultInstance), - partitioning = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest._typemapper_partitioning.toCustom(__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage]).getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.defaultInstance)) + edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest( + tag = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.PhysicalLink]).getOrElse(edu.uci.ics.amber.engine.common.PhysicalLink.defaultInstance), + partitioning = edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest._typemapper_partitioning.toCustom(__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage]).getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.defaultInstance)) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(28) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(28) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(28) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(28) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.workflow.PhysicalLink - case 2 => __out = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage + case 1 => __out = edu.uci.ics.amber.engine.common.PhysicalLink + case 2 => __out = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest( - tag = edu.uci.ics.amber.engine.common.workflow.PhysicalLink.defaultInstance, - partitioning = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest._typemapper_partitioning.toCustom(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.defaultInstance) + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest( + tag = edu.uci.ics.amber.engine.common.PhysicalLink.defaultInstance, + partitioning = edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest._typemapper_partitioning.toCustom(edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.defaultInstance) ) - implicit class AddPartitioningRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest](_l) { - def tag: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PhysicalLink] = field(_.tag)((c_, f_) => c_.copy(tag = f_)) - def partitioning: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning] = field(_.partitioning)((c_, f_) => c_.copy(partitioning = f_)) + implicit class AddPartitioningRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest](_l) { + def tag: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PhysicalLink] = field(_.tag)((c_, f_) => c_.copy(tag = f_)) + def partitioning: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning] = field(_.partitioning)((c_, f_) => c_.copy(partitioning = f_)) } final val TAG_FIELD_NUMBER = 1 final val PARTITIONING_FIELD_NUMBER = 2 @transient - private[controlcommands] val _typemapper_partitioning: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning]] + private[rpc] val _typemapper_partitioning: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage, edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage, edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning]] def of( - tag: edu.uci.ics.amber.engine.common.workflow.PhysicalLink, - partitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest( + tag: edu.uci.ics.amber.engine.common.PhysicalLink, + partitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest( tag, partitioning ) @@ -2699,10 +2699,10 @@ object AddPartitioningRequest extends scalapb.GeneratedMessageCompanion[edu.uci. @SerialVersionUID(0L) final case class AssignPortRequest( - portId: edu.uci.ics.amber.engine.common.workflow.PortIdentity, + portId: edu.uci.ics.amber.engine.common.PortIdentity, input: _root_.scala.Boolean, schema: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, _root_.scala.Predef.String] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[AssignPortRequest] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[AssignPortRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -2722,7 +2722,7 @@ final case class AssignPortRequest( } }; schema.foreach { __item => - val __value = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest._typemapper_schema.toBase(__item) + val __value = edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest._typemapper_schema.toBase(__item) __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } __size @@ -2752,13 +2752,13 @@ final case class AssignPortRequest( } }; schema.foreach { __v => - val __m = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest._typemapper_schema.toBase(__v) + val __m = edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest._typemapper_schema.toBase(__v) _output__.writeTag(3, 2) _output__.writeUInt32NoTag(__m.serializedSize) __m.writeTo(_output__) }; } - def withPortId(__v: edu.uci.ics.amber.engine.common.workflow.PortIdentity): AssignPortRequest = copy(portId = __v) + def withPortId(__v: edu.uci.ics.amber.engine.common.PortIdentity): AssignPortRequest = copy(portId = __v) def withInput(__v: _root_.scala.Boolean): AssignPortRequest = copy(input = __v) def clearSchema = copy(schema = _root_.scala.collection.immutable.Map.empty) def addSchema(__vs: (_root_.scala.Predef.String, _root_.scala.Predef.String) *): AssignPortRequest = addAllSchema(__vs) @@ -2768,13 +2768,13 @@ final case class AssignPortRequest( (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = portId - if (__t != edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance) __t else null } case 2 => { val __t = input if (__t != false) __t else null } - case 3 => schema.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest._typemapper_schema.toBase(_)).toSeq + case 3 => schema.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest._typemapper_schema.toBase(_)).toSeq } } def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = { @@ -2782,18 +2782,18 @@ final case class AssignPortRequest( (__field.number: @_root_.scala.unchecked) match { case 1 => portId.toPMessage case 2 => _root_.scalapb.descriptors.PBoolean(input) - case 3 => _root_.scalapb.descriptors.PRepeated(schema.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest._typemapper_schema.toBase(_).toPMessage).toVector) + case 3 => _root_.scalapb.descriptors.PRepeated(schema.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest._typemapper_schema.toBase(_).toPMessage).toVector) } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.type = edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest]) } -object AssignPortRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest = { - var __portId: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflow.PortIdentity] = _root_.scala.None +object AssignPortRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest = { + var __portId: _root_.scala.Option[edu.uci.ics.amber.engine.common.PortIdentity] = _root_.scala.None var __input: _root_.scala.Boolean = false val __schema: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, _root_.scala.Predef.String), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, _root_.scala.Predef.String]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, _root_.scala.Predef.String] var _done__ = false @@ -2802,47 +2802,47 @@ object AssignPortRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.a _tag__ match { case 0 => _done__ = true case 10 => - __portId = _root_.scala.Some(__portId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflow.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __portId = _root_.scala.Some(__portId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 16 => __input = _input__.readBool() case 26 => - __schema += edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest._typemapper_schema.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry](_input__)) + __schema += edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest._typemapper_schema.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry](_input__)) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest( - portId = __portId.getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest( + portId = __portId.getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance), input = __input, schema = __schema.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest( - portId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.workflow.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest( + portId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance), input = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Boolean]).getOrElse(false), - schema = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest._typemapper_schema.toCustom(_)).toMap + schema = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest._typemapper_schema.toCustom(_)).toMap ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(29) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(29) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(29) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(29) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.workflow.PortIdentity - case 3 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry + case 1 => __out = edu.uci.ics.amber.engine.common.PortIdentity + case 3 => __out = edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry + _root_.edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry ) def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest( - portId = edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance, + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest( + portId = edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance, input = false, schema = _root_.scala.collection.immutable.Map.empty ) @@ -2916,13 +2916,13 @@ object AssignPortRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.a } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry + def companion: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry.type = edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry]) } - object SchemaEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry = { + object SchemaEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry = { var __key: _root_.scala.Predef.String = "" var __value: _root_.scala.Predef.String = "" var _done__ = false @@ -2937,63 +2937,63 @@ object AssignPortRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.a case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry( + edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry( key = __key, value = __value ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry( + edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry( key = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.javaDescriptor.getNestedTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.scalaDescriptor.nestedMessages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.javaDescriptor.getNestedTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.scalaDescriptor.nestedMessages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry( key = "", value = "" ) - implicit class SchemaEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry](_l) { + implicit class SchemaEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry](_l) { def key: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.key)((c_, f_) => c_.copy(key = f_)) def value: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.value)((c_, f_) => c_.copy(value = f_)) } final val KEY_FIELD_NUMBER = 1 final val VALUE_FIELD_NUMBER = 2 @transient - implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)] = - _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)](__m => (__m.key, __m.value))(__p => edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry(__p._1, __p._2)) + implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)] = + _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)](__m => (__m.key, __m.value))(__p => edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry(__p._1, __p._2)) def of( key: _root_.scala.Predef.String, value: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry = _root_.edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry( key, value ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry]) } - implicit class AssignPortRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest](_l) { - def portId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PortIdentity] = field(_.portId)((c_, f_) => c_.copy(portId = f_)) + implicit class AssignPortRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest](_l) { + def portId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PortIdentity] = field(_.portId)((c_, f_) => c_.copy(portId = f_)) def input: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Boolean] = field(_.input)((c_, f_) => c_.copy(input = f_)) def schema: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, _root_.scala.Predef.String]] = field(_.schema)((c_, f_) => c_.copy(schema = f_)) } - final val PORTID_FIELD_NUMBER = 1 + final val PORT_ID_FIELD_NUMBER = 1 final val INPUT_FIELD_NUMBER = 2 final val SCHEMA_FIELD_NUMBER = 3 @transient - private[controlcommands] val _typemapper_schema: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest.SchemaEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)]] + private[rpc] val _typemapper_schema: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest.SchemaEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)]] def of( - portId: edu.uci.ics.amber.engine.common.workflow.PortIdentity, + portId: edu.uci.ics.amber.engine.common.PortIdentity, input: _root_.scala.Boolean, schema: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, _root_.scala.Predef.String] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest( portId, input, schema @@ -3003,9 +3003,9 @@ object AssignPortRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.a @SerialVersionUID(0L) final case class FinalizeCheckpointRequest( - checkpointId: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity, + checkpointId: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity, writeTo: _root_.scala.Predef.String - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[FinalizeCheckpointRequest] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[FinalizeCheckpointRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -3051,13 +3051,13 @@ final case class FinalizeCheckpointRequest( } }; } - def withCheckpointId(__v: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity): FinalizeCheckpointRequest = copy(checkpointId = __v) + def withCheckpointId(__v: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity): FinalizeCheckpointRequest = copy(checkpointId = __v) def withWriteTo(__v: _root_.scala.Predef.String): FinalizeCheckpointRequest = copy(writeTo = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = checkpointId - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance) __t else null } case 2 => { val __t = writeTo @@ -3073,14 +3073,14 @@ final case class FinalizeCheckpointRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest.type = edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest]) } -object FinalizeCheckpointRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest = { - var __checkpointId: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] = _root_.scala.None +object FinalizeCheckpointRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest = { + var __checkpointId: _root_.scala.Option[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] = _root_.scala.None var __writeTo: _root_.scala.Predef.String = "" var _done__ = false while (!_done__) { @@ -3088,51 +3088,51 @@ object FinalizeCheckpointRequest extends scalapb.GeneratedMessageCompanion[edu.u _tag__ match { case 0 => _done__ = true case 10 => - __checkpointId = _root_.scala.Some(__checkpointId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __checkpointId = _root_.scala.Some(__checkpointId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => __writeTo = _input__.readStringRequireUtf8() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest( - checkpointId = __checkpointId.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest( + checkpointId = __checkpointId.getOrElse(edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance), writeTo = __writeTo ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest( - checkpointId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest( + checkpointId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance), writeTo = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(30) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(30) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(30) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(30) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.ChannelMarkerIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest( - checkpointId = edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance, + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest( + checkpointId = edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance, writeTo = "" ) - implicit class FinalizeCheckpointRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest](_l) { - def checkpointId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] = field(_.checkpointId)((c_, f_) => c_.copy(checkpointId = f_)) + implicit class FinalizeCheckpointRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest](_l) { + def checkpointId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] = field(_.checkpointId)((c_, f_) => c_.copy(checkpointId = f_)) def writeTo: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.writeTo)((c_, f_) => c_.copy(writeTo = f_)) } - final val CHECKPOINTID_FIELD_NUMBER = 1 - final val WRITETO_FIELD_NUMBER = 2 + final val CHECKPOINT_ID_FIELD_NUMBER = 1 + final val WRITE_TO_FIELD_NUMBER = 2 def of( - checkpointId: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity, + checkpointId: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity, writeTo: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest( checkpointId, writeTo ) @@ -3145,7 +3145,7 @@ final case class InitializeExecutorRequest( opExecInitInfo: com.google.protobuf.any.Any, isSource: _root_.scala.Boolean, language: _root_.scala.Predef.String - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[InitializeExecutorRequest] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[InitializeExecutorRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -3251,13 +3251,13 @@ final case class InitializeExecutorRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest.type = edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest]) } -object InitializeExecutorRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest = { +object InitializeExecutorRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest = { var __totalWorkerCount: _root_.scala.Int = 0 var __opExecInitInfo: _root_.scala.Option[com.google.protobuf.any.Any] = _root_.scala.None var __isSource: _root_.scala.Boolean = false @@ -3278,17 +3278,17 @@ object InitializeExecutorRequest extends scalapb.GeneratedMessageCompanion[edu.u case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest( + edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest( totalWorkerCount = __totalWorkerCount, opExecInitInfo = __opExecInitInfo.getOrElse(com.google.protobuf.any.Any.defaultInstance), isSource = __isSource, language = __language ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest( + edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest( totalWorkerCount = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Int]).getOrElse(0), opExecInitInfo = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[com.google.protobuf.any.Any]).getOrElse(com.google.protobuf.any.Any.defaultInstance), isSource = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Boolean]).getOrElse(false), @@ -3296,8 +3296,8 @@ object InitializeExecutorRequest extends scalapb.GeneratedMessageCompanion[edu.u ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(31) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(31) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(31) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(31) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { @@ -3307,28 +3307,28 @@ object InitializeExecutorRequest extends scalapb.GeneratedMessageCompanion[edu.u } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest( totalWorkerCount = 0, opExecInitInfo = com.google.protobuf.any.Any.defaultInstance, isSource = false, language = "" ) - implicit class InitializeExecutorRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest](_l) { + implicit class InitializeExecutorRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest](_l) { def totalWorkerCount: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.totalWorkerCount)((c_, f_) => c_.copy(totalWorkerCount = f_)) def opExecInitInfo: _root_.scalapb.lenses.Lens[UpperPB, com.google.protobuf.any.Any] = field(_.opExecInitInfo)((c_, f_) => c_.copy(opExecInitInfo = f_)) def isSource: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Boolean] = field(_.isSource)((c_, f_) => c_.copy(isSource = f_)) def language: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.language)((c_, f_) => c_.copy(language = f_)) } - final val TOTALWORKERCOUNT_FIELD_NUMBER = 1 - final val OPEXECINITINFO_FIELD_NUMBER = 2 - final val ISSOURCE_FIELD_NUMBER = 3 + final val TOTAL_WORKER_COUNT_FIELD_NUMBER = 1 + final val OP_EXEC_INIT_INFO_FIELD_NUMBER = 2 + final val IS_SOURCE_FIELD_NUMBER = 3 final val LANGUAGE_FIELD_NUMBER = 4 def of( totalWorkerCount: _root_.scala.Int, opExecInitInfo: com.google.protobuf.any.Any, isSource: _root_.scala.Boolean, language: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest( totalWorkerCount, opExecInitInfo, isSource, @@ -3339,10 +3339,10 @@ object InitializeExecutorRequest extends scalapb.GeneratedMessageCompanion[edu.u @SerialVersionUID(0L) final case class UpdateExecutorRequest( - targetOpId: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity, + targetOpId: edu.uci.ics.amber.engine.common.PhysicalOpIdentity, newExecutor: com.google.protobuf.any.Any, stateTransferFunc: _root_.scala.Option[com.google.protobuf.any.Any] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[UpdateExecutorRequest] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[UpdateExecutorRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -3400,7 +3400,7 @@ final case class UpdateExecutorRequest( __m.writeTo(_output__) }; } - def withTargetOpId(__v: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity): UpdateExecutorRequest = copy(targetOpId = __v) + def withTargetOpId(__v: edu.uci.ics.amber.engine.common.PhysicalOpIdentity): UpdateExecutorRequest = copy(targetOpId = __v) def withNewExecutor(__v: com.google.protobuf.any.Any): UpdateExecutorRequest = copy(newExecutor = __v) def getStateTransferFunc: com.google.protobuf.any.Any = stateTransferFunc.getOrElse(com.google.protobuf.any.Any.defaultInstance) def clearStateTransferFunc: UpdateExecutorRequest = copy(stateTransferFunc = _root_.scala.None) @@ -3409,7 +3409,7 @@ final case class UpdateExecutorRequest( (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = targetOpId - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance) __t else null } case 2 => { val __t = newExecutor @@ -3427,14 +3427,14 @@ final case class UpdateExecutorRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest.type = edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest]) } -object UpdateExecutorRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest = { - var __targetOpId: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] = _root_.scala.None +object UpdateExecutorRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest = { + var __targetOpId: _root_.scala.Option[edu.uci.ics.amber.engine.common.PhysicalOpIdentity] = _root_.scala.None var __newExecutor: _root_.scala.Option[com.google.protobuf.any.Any] = _root_.scala.None var __stateTransferFunc: _root_.scala.Option[com.google.protobuf.any.Any] = _root_.scala.None var _done__ = false @@ -3443,7 +3443,7 @@ object UpdateExecutorRequest extends scalapb.GeneratedMessageCompanion[edu.uci.i _tag__ match { case 0 => _done__ = true case 10 => - __targetOpId = _root_.scala.Some(__targetOpId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __targetOpId = _root_.scala.Some(__targetOpId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PhysicalOpIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => __newExecutor = _root_.scala.Some(__newExecutor.fold(_root_.scalapb.LiteParser.readMessage[com.google.protobuf.any.Any](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 26 => @@ -3451,28 +3451,28 @@ object UpdateExecutorRequest extends scalapb.GeneratedMessageCompanion[edu.uci.i case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest( - targetOpId = __targetOpId.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest( + targetOpId = __targetOpId.getOrElse(edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance), newExecutor = __newExecutor.getOrElse(com.google.protobuf.any.Any.defaultInstance), stateTransferFunc = __stateTransferFunc ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest( - targetOpId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest( + targetOpId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]).getOrElse(edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance), newExecutor = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[com.google.protobuf.any.Any]).getOrElse(com.google.protobuf.any.Any.defaultInstance), stateTransferFunc = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).flatMap(_.as[_root_.scala.Option[com.google.protobuf.any.Any]]) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(32) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(32) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(32) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(32) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.PhysicalOpIdentity case 2 => __out = com.google.protobuf.any.Any case 3 => __out = com.google.protobuf.any.Any } @@ -3480,25 +3480,25 @@ object UpdateExecutorRequest extends scalapb.GeneratedMessageCompanion[edu.uci.i } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest( - targetOpId = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance, + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest( + targetOpId = edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance, newExecutor = com.google.protobuf.any.Any.defaultInstance, stateTransferFunc = _root_.scala.None ) - implicit class UpdateExecutorRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest](_l) { - def targetOpId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] = field(_.targetOpId)((c_, f_) => c_.copy(targetOpId = f_)) + implicit class UpdateExecutorRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest](_l) { + def targetOpId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PhysicalOpIdentity] = field(_.targetOpId)((c_, f_) => c_.copy(targetOpId = f_)) def newExecutor: _root_.scalapb.lenses.Lens[UpperPB, com.google.protobuf.any.Any] = field(_.newExecutor)((c_, f_) => c_.copy(newExecutor = f_)) def stateTransferFunc: _root_.scalapb.lenses.Lens[UpperPB, com.google.protobuf.any.Any] = field(_.getStateTransferFunc)((c_, f_) => c_.copy(stateTransferFunc = Option(f_))) def optionalStateTransferFunc: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[com.google.protobuf.any.Any]] = field(_.stateTransferFunc)((c_, f_) => c_.copy(stateTransferFunc = f_)) } - final val TARGETOPID_FIELD_NUMBER = 1 - final val NEWEXECUTOR_FIELD_NUMBER = 2 - final val STATETRANSFERFUNC_FIELD_NUMBER = 3 + final val TARGET_OP_ID_FIELD_NUMBER = 1 + final val NEW_EXECUTOR_FIELD_NUMBER = 2 + final val STATE_TRANSFER_FUNC_FIELD_NUMBER = 3 def of( - targetOpId: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity, + targetOpId: edu.uci.ics.amber.engine.common.PhysicalOpIdentity, newExecutor: com.google.protobuf.any.Any, stateTransferFunc: _root_.scala.Option[com.google.protobuf.any.Any] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.UpdateExecutorRequest( targetOpId, newExecutor, stateTransferFunc @@ -3508,20 +3508,20 @@ object UpdateExecutorRequest extends scalapb.GeneratedMessageCompanion[edu.uci.i @SerialVersionUID(0L) final case class EmptyRequest( - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[EmptyRequest] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[EmptyRequest] { final override def serializedSize: _root_.scala.Int = 0 def writeTo(`_output__`: _root_.com.google.protobuf.CodedOutputStream): _root_.scala.Unit = { } def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = throw new MatchError(__fieldNumber) def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = throw new MatchError(__field) def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest.type = edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) } -object EmptyRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest = { +object EmptyRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest = { var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -3530,36 +3530,36 @@ object EmptyRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber. case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest( + edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest( ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest( + edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest( ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(1) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(1) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(1) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(1) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest( ) - implicit class EmptyRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest](_l) { + implicit class EmptyRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest](_l) { } def of( - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest( ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) } @SerialVersionUID(0L) final case class PrepareCheckpointRequest( - checkpointId: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity, + checkpointId: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity, estimationOnly: _root_.scala.Boolean - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[PrepareCheckpointRequest] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[PrepareCheckpointRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -3605,13 +3605,13 @@ final case class PrepareCheckpointRequest( } }; } - def withCheckpointId(__v: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity): PrepareCheckpointRequest = copy(checkpointId = __v) + def withCheckpointId(__v: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity): PrepareCheckpointRequest = copy(checkpointId = __v) def withEstimationOnly(__v: _root_.scala.Boolean): PrepareCheckpointRequest = copy(estimationOnly = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = checkpointId - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance) __t else null } case 2 => { val __t = estimationOnly @@ -3627,14 +3627,14 @@ final case class PrepareCheckpointRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest.type = edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest]) } -object PrepareCheckpointRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest = { - var __checkpointId: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] = _root_.scala.None +object PrepareCheckpointRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest = { + var __checkpointId: _root_.scala.Option[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] = _root_.scala.None var __estimationOnly: _root_.scala.Boolean = false var _done__ = false while (!_done__) { @@ -3642,51 +3642,51 @@ object PrepareCheckpointRequest extends scalapb.GeneratedMessageCompanion[edu.uc _tag__ match { case 0 => _done__ = true case 10 => - __checkpointId = _root_.scala.Some(__checkpointId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __checkpointId = _root_.scala.Some(__checkpointId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 16 => __estimationOnly = _input__.readBool() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest( - checkpointId = __checkpointId.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest( + checkpointId = __checkpointId.getOrElse(edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance), estimationOnly = __estimationOnly ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest( - checkpointId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest( + checkpointId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance), estimationOnly = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Boolean]).getOrElse(false) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(33) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(33) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(33) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(33) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.ChannelMarkerIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest( - checkpointId = edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.defaultInstance, + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest( + checkpointId = edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.defaultInstance, estimationOnly = false ) - implicit class PrepareCheckpointRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest](_l) { - def checkpointId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] = field(_.checkpointId)((c_, f_) => c_.copy(checkpointId = f_)) + implicit class PrepareCheckpointRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest](_l) { + def checkpointId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] = field(_.checkpointId)((c_, f_) => c_.copy(checkpointId = f_)) def estimationOnly: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Boolean] = field(_.estimationOnly)((c_, f_) => c_.copy(estimationOnly = f_)) } - final val CHECKPOINTID_FIELD_NUMBER = 1 - final val ESTIMATIONONLY_FIELD_NUMBER = 2 + final val CHECKPOINT_ID_FIELD_NUMBER = 1 + final val ESTIMATION_ONLY_FIELD_NUMBER = 2 def of( - checkpointId: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity, + checkpointId: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity, estimationOnly: _root_.scala.Boolean - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest( checkpointId, estimationOnly ) @@ -3695,8 +3695,8 @@ object PrepareCheckpointRequest extends scalapb.GeneratedMessageCompanion[edu.uc @SerialVersionUID(0L) final case class QueryStatisticsRequest( - filterByWorkers: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[QueryStatisticsRequest] { + filterByWorkers: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[QueryStatisticsRequest] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -3725,9 +3725,9 @@ final case class QueryStatisticsRequest( }; } def clearFilterByWorkers = copy(filterByWorkers = _root_.scala.Seq.empty) - def addFilterByWorkers(__vs: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity *): QueryStatisticsRequest = addAllFilterByWorkers(__vs) - def addAllFilterByWorkers(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]): QueryStatisticsRequest = copy(filterByWorkers = filterByWorkers ++ __vs) - def withFilterByWorkers(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]): QueryStatisticsRequest = copy(filterByWorkers = __v) + def addFilterByWorkers(__vs: edu.uci.ics.amber.engine.common.ActorVirtualIdentity *): QueryStatisticsRequest = addAllFilterByWorkers(__vs) + def addAllFilterByWorkers(__vs: Iterable[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]): QueryStatisticsRequest = copy(filterByWorkers = filterByWorkers ++ __vs) + def withFilterByWorkers(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]): QueryStatisticsRequest = copy(filterByWorkers = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => filterByWorkers @@ -3740,57 +3740,57 @@ final case class QueryStatisticsRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest.type = edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest]) } -object QueryStatisticsRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest = { - val __filterByWorkers: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] +object QueryStatisticsRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest = { + val __filterByWorkers: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __filterByWorkers += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__) + __filterByWorkers += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest( + edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest( filterByWorkers = __filterByWorkers.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest( - filterByWorkers = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]]).getOrElse(_root_.scala.Seq.empty) + edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest( + filterByWorkers = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(34) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(34) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(34) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(34) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest( filterByWorkers = _root_.scala.Seq.empty ) - implicit class QueryStatisticsRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest](_l) { - def filterByWorkers: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]] = field(_.filterByWorkers)((c_, f_) => c_.copy(filterByWorkers = f_)) + implicit class QueryStatisticsRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest](_l) { + def filterByWorkers: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]] = field(_.filterByWorkers)((c_, f_) => c_.copy(filterByWorkers = f_)) } - final val FILTERBYWORKERS_FIELD_NUMBER = 1 + final val FILTER_BY_WORKERS_FIELD_NUMBER = 1 def of( - filterByWorkers: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest( + filterByWorkers: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest( filterByWorkers ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest]) @@ -3802,8 +3802,8 @@ object QueryStatisticsRequest extends scalapb.GeneratedMessageCompanion[edu.uci. final case class Ping( i: _root_.scala.Int, end: _root_.scala.Int, - to: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Ping] { + to: edu.uci.ics.amber.engine.common.ActorVirtualIdentity + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Ping] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -3864,7 +3864,7 @@ final case class Ping( } def withI(__v: _root_.scala.Int): Ping = copy(i = __v) def withEnd(__v: _root_.scala.Int): Ping = copy(end = __v) - def withTo(__v: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity): Ping = copy(to = __v) + def withTo(__v: edu.uci.ics.amber.engine.common.ActorVirtualIdentity): Ping = copy(to = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -3877,7 +3877,7 @@ final case class Ping( } case 3 => { val __t = to - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) __t else null } } } @@ -3890,16 +3890,16 @@ final case class Ping( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping + def companion: edu.uci.ics.amber.engine.architecture.rpc.Ping.type = edu.uci.ics.amber.engine.architecture.rpc.Ping // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.Ping]) } -object Ping extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping = { +object Ping extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Ping] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Ping] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.Ping = { var __i: _root_.scala.Int = 0 var __end: _root_.scala.Int = 0 - var __to: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = _root_.scala.None + var __to: _root_.scala.Option[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -3910,46 +3910,46 @@ object Ping extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.a case 16 => __end = _input__.readInt32() case 26 => - __to = _root_.scala.Some(__to.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __to = _root_.scala.Some(__to.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping( + edu.uci.ics.amber.engine.architecture.rpc.Ping( i = __i, end = __end, - to = __to.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) + to = __to.getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.Ping] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping( + edu.uci.ics.amber.engine.architecture.rpc.Ping( i = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Int]).getOrElse(0), end = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Int]).getOrElse(0), - to = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) + to = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(17) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(17) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(17) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(17) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 3 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity + case 3 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.Ping( i = 0, end = 0, - to = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance + to = edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance ) - implicit class PingLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping](_l) { + implicit class PingLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Ping]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Ping](_l) { def i: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.i)((c_, f_) => c_.copy(i = f_)) def end: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.end)((c_, f_) => c_.copy(end = f_)) - def to: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = field(_.to)((c_, f_) => c_.copy(to = f_)) + def to: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = field(_.to)((c_, f_) => c_.copy(to = f_)) } final val I_FIELD_NUMBER = 1 final val END_FIELD_NUMBER = 2 @@ -3957,8 +3957,8 @@ object Ping extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.a def of( i: _root_.scala.Int, end: _root_.scala.Int, - to: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping( + to: edu.uci.ics.amber.engine.common.ActorVirtualIdentity + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.Ping = _root_.edu.uci.ics.amber.engine.architecture.rpc.Ping( i, end, to @@ -3972,8 +3972,8 @@ object Ping extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.a final case class Pong( i: _root_.scala.Int, end: _root_.scala.Int, - to: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Pong] { + to: edu.uci.ics.amber.engine.common.ActorVirtualIdentity + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Pong] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -4034,7 +4034,7 @@ final case class Pong( } def withI(__v: _root_.scala.Int): Pong = copy(i = __v) def withEnd(__v: _root_.scala.Int): Pong = copy(end = __v) - def withTo(__v: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity): Pong = copy(to = __v) + def withTo(__v: edu.uci.ics.amber.engine.common.ActorVirtualIdentity): Pong = copy(to = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -4047,7 +4047,7 @@ final case class Pong( } case 3 => { val __t = to - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) __t else null } } } @@ -4060,16 +4060,16 @@ final case class Pong( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong + def companion: edu.uci.ics.amber.engine.architecture.rpc.Pong.type = edu.uci.ics.amber.engine.architecture.rpc.Pong // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.Pong]) } -object Pong extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong = { +object Pong extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Pong] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Pong] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.Pong = { var __i: _root_.scala.Int = 0 var __end: _root_.scala.Int = 0 - var __to: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = _root_.scala.None + var __to: _root_.scala.Option[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -4080,46 +4080,46 @@ object Pong extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.a case 16 => __end = _input__.readInt32() case 26 => - __to = _root_.scala.Some(__to.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __to = _root_.scala.Some(__to.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong( + edu.uci.ics.amber.engine.architecture.rpc.Pong( i = __i, end = __end, - to = __to.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) + to = __to.getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.Pong] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong( + edu.uci.ics.amber.engine.architecture.rpc.Pong( i = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Int]).getOrElse(0), end = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Int]).getOrElse(0), - to = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) + to = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(18) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(18) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(18) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(18) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 3 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity + case 3 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.Pong( i = 0, end = 0, - to = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance + to = edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance ) - implicit class PongLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong](_l) { + implicit class PongLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Pong]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Pong](_l) { def i: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.i)((c_, f_) => c_.copy(i = f_)) def end: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.end)((c_, f_) => c_.copy(end = f_)) - def to: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = field(_.to)((c_, f_) => c_.copy(to = f_)) + def to: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = field(_.to)((c_, f_) => c_.copy(to = f_)) } final val I_FIELD_NUMBER = 1 final val END_FIELD_NUMBER = 2 @@ -4127,8 +4127,8 @@ object Pong extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.a def of( i: _root_.scala.Int, end: _root_.scala.Int, - to: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong( + to: edu.uci.ics.amber.engine.common.ActorVirtualIdentity + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.Pong = _root_.edu.uci.ics.amber.engine.architecture.rpc.Pong( i, end, to @@ -4141,7 +4141,7 @@ object Pong extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.a @SerialVersionUID(0L) final case class Nested( k: _root_.scala.Int - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Nested] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Nested] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -4188,13 +4188,13 @@ final case class Nested( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested + def companion: edu.uci.ics.amber.engine.architecture.rpc.Nested.type = edu.uci.ics.amber.engine.architecture.rpc.Nested // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.Nested]) } -object Nested extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested = { +object Nested extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Nested] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Nested] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.Nested = { var __k: _root_.scala.Int = 0 var _done__ = false while (!_done__) { @@ -4206,33 +4206,33 @@ object Nested extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested( + edu.uci.ics.amber.engine.architecture.rpc.Nested( k = __k ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.Nested] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested( + edu.uci.ics.amber.engine.architecture.rpc.Nested( k = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Int]).getOrElse(0) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(20) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(20) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(20) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(20) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.Nested( k = 0 ) - implicit class NestedLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested](_l) { + implicit class NestedLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Nested]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Nested](_l) { def k: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.k)((c_, f_) => c_.copy(k = f_)) } final val K_FIELD_NUMBER = 1 def of( k: _root_.scala.Int - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.Nested = _root_.edu.uci.ics.amber.engine.architecture.rpc.Nested( k ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Nested]) @@ -4243,7 +4243,7 @@ object Nested extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine @SerialVersionUID(0L) final case class Pass( value: _root_.scala.Predef.String - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Pass] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Pass] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -4290,13 +4290,13 @@ final case class Pass( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass + def companion: edu.uci.ics.amber.engine.architecture.rpc.Pass.type = edu.uci.ics.amber.engine.architecture.rpc.Pass // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.Pass]) } -object Pass extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass = { +object Pass extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Pass] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Pass] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.Pass = { var __value: _root_.scala.Predef.String = "" var _done__ = false while (!_done__) { @@ -4308,33 +4308,33 @@ object Pass extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.a case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass( + edu.uci.ics.amber.engine.architecture.rpc.Pass( value = __value ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.Pass] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass( + edu.uci.ics.amber.engine.architecture.rpc.Pass( value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(19) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(19) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(19) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(19) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.Pass( value = "" ) - implicit class PassLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass](_l) { + implicit class PassLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Pass]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Pass](_l) { def value: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.value)((c_, f_) => c_.copy(value = f_)) } final val VALUE_FIELD_NUMBER = 1 def of( value: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.Pass = _root_.edu.uci.ics.amber.engine.architecture.rpc.Pass( value ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Pass]) @@ -4344,20 +4344,20 @@ object Pass extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.a */ @SerialVersionUID(0L) final case class ErrorCommand( - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[ErrorCommand] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[ErrorCommand] { final override def serializedSize: _root_.scala.Int = 0 def writeTo(`_output__`: _root_.com.google.protobuf.CodedOutputStream): _root_.scala.Unit = { } def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = throw new MatchError(__fieldNumber) def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = throw new MatchError(__field) def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand + def companion: edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand.type = edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand]) } -object ErrorCommand extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand = { +object ErrorCommand extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand = { var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -4366,27 +4366,27 @@ object ErrorCommand extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber. case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand( + edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand( ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand( + edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand( ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(22) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(22) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(22) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(22) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand( ) - implicit class ErrorCommandLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand](_l) { + implicit class ErrorCommandLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand](_l) { } def of( - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand = _root_.edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand( ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand]) } @@ -4396,7 +4396,7 @@ object ErrorCommand extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber. @SerialVersionUID(0L) final case class Recursion( i: _root_.scala.Int - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Recursion] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Recursion] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -4443,13 +4443,13 @@ final case class Recursion( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion + def companion: edu.uci.ics.amber.engine.architecture.rpc.Recursion.type = edu.uci.ics.amber.engine.architecture.rpc.Recursion // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.Recursion]) } -object Recursion extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion = { +object Recursion extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Recursion] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Recursion] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.Recursion = { var __i: _root_.scala.Int = 0 var _done__ = false while (!_done__) { @@ -4461,33 +4461,33 @@ object Recursion extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.eng case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion( + edu.uci.ics.amber.engine.architecture.rpc.Recursion( i = __i ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.Recursion] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion( + edu.uci.ics.amber.engine.architecture.rpc.Recursion( i = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Int]).getOrElse(0) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(26) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(26) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(26) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(26) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.Recursion( i = 0 ) - implicit class RecursionLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion](_l) { + implicit class RecursionLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Recursion]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Recursion](_l) { def i: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.i)((c_, f_) => c_.copy(i = f_)) } final val I_FIELD_NUMBER = 1 def of( i: _root_.scala.Int - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.Recursion = _root_.edu.uci.ics.amber.engine.architecture.rpc.Recursion( i ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Recursion]) @@ -4497,8 +4497,8 @@ object Recursion extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.eng */ @SerialVersionUID(0L) final case class Collect( - workers: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Collect] { + workers: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Collect] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -4527,9 +4527,9 @@ final case class Collect( }; } def clearWorkers = copy(workers = _root_.scala.Seq.empty) - def addWorkers(__vs: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity *): Collect = addAllWorkers(__vs) - def addAllWorkers(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]): Collect = copy(workers = workers ++ __vs) - def withWorkers(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]): Collect = copy(workers = __v) + def addWorkers(__vs: edu.uci.ics.amber.engine.common.ActorVirtualIdentity *): Collect = addAllWorkers(__vs) + def addAllWorkers(__vs: Iterable[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]): Collect = copy(workers = workers ++ __vs) + def withWorkers(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]): Collect = copy(workers = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => workers @@ -4542,57 +4542,57 @@ final case class Collect( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect + def companion: edu.uci.ics.amber.engine.architecture.rpc.Collect.type = edu.uci.ics.amber.engine.architecture.rpc.Collect // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.Collect]) } -object Collect extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect = { - val __workers: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] +object Collect extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Collect] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Collect] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.Collect = { + val __workers: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __workers += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__) + __workers += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect( + edu.uci.ics.amber.engine.architecture.rpc.Collect( workers = __workers.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.Collect] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect( - workers = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]]).getOrElse(_root_.scala.Seq.empty) + edu.uci.ics.amber.engine.architecture.rpc.Collect( + workers = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(23) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(23) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(23) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(23) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.Collect( workers = _root_.scala.Seq.empty ) - implicit class CollectLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect](_l) { - def workers: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]] = field(_.workers)((c_, f_) => c_.copy(workers = f_)) + implicit class CollectLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Collect]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Collect](_l) { + def workers: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]] = field(_.workers)((c_, f_) => c_.copy(workers = f_)) } final val WORKERS_FIELD_NUMBER = 1 def of( - workers: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect( + workers: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.Collect = _root_.edu.uci.ics.amber.engine.architecture.rpc.Collect( workers ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Collect]) @@ -4602,20 +4602,20 @@ object Collect extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engin */ @SerialVersionUID(0L) final case class GenerateNumber( - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[GenerateNumber] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[GenerateNumber] { final override def serializedSize: _root_.scala.Int = 0 def writeTo(`_output__`: _root_.com.google.protobuf.CodedOutputStream): _root_.scala.Unit = { } def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = throw new MatchError(__fieldNumber) def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = throw new MatchError(__field) def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber + def companion: edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber.type = edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber]) } -object GenerateNumber extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber = { +object GenerateNumber extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber = { var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -4624,27 +4624,27 @@ object GenerateNumber extends scalapb.GeneratedMessageCompanion[edu.uci.ics.ambe case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber( + edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber( ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber( + edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber( ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(24) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(24) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(24) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(24) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber( ) - implicit class GenerateNumberLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber](_l) { + implicit class GenerateNumberLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber](_l) { } def of( - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber = _root_.edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber( ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber]) } @@ -4653,8 +4653,8 @@ object GenerateNumber extends scalapb.GeneratedMessageCompanion[edu.uci.ics.ambe */ @SerialVersionUID(0L) final case class MultiCall( - seq: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[MultiCall] { + seq: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[MultiCall] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -4683,9 +4683,9 @@ final case class MultiCall( }; } def clearSeq = copy(seq = _root_.scala.Seq.empty) - def addSeq(__vs: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity *): MultiCall = addAllSeq(__vs) - def addAllSeq(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]): MultiCall = copy(seq = seq ++ __vs) - def withSeq(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]): MultiCall = copy(seq = __v) + def addSeq(__vs: edu.uci.ics.amber.engine.common.ActorVirtualIdentity *): MultiCall = addAllSeq(__vs) + def addAllSeq(__vs: Iterable[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]): MultiCall = copy(seq = seq ++ __vs) + def withSeq(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]): MultiCall = copy(seq = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => seq @@ -4698,57 +4698,57 @@ final case class MultiCall( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall + def companion: edu.uci.ics.amber.engine.architecture.rpc.MultiCall.type = edu.uci.ics.amber.engine.architecture.rpc.MultiCall // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.MultiCall]) } -object MultiCall extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall = { - val __seq: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] +object MultiCall extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.MultiCall] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.MultiCall] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.MultiCall = { + val __seq: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __seq += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__) + __seq += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall( + edu.uci.ics.amber.engine.architecture.rpc.MultiCall( seq = __seq.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.MultiCall] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall( - seq = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]]).getOrElse(_root_.scala.Seq.empty) + edu.uci.ics.amber.engine.architecture.rpc.MultiCall( + seq = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(21) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(21) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(21) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(21) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.MultiCall( seq = _root_.scala.Seq.empty ) - implicit class MultiCallLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall](_l) { - def seq: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]] = field(_.seq)((c_, f_) => c_.copy(seq = f_)) + implicit class MultiCallLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.MultiCall]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.MultiCall](_l) { + def seq: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]] = field(_.seq)((c_, f_) => c_.copy(seq = f_)) } final val SEQ_FIELD_NUMBER = 1 def of( - seq: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall( + seq: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.MultiCall = _root_.edu.uci.ics.amber.engine.architecture.rpc.MultiCall( seq ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.MultiCall]) @@ -4758,8 +4758,8 @@ object MultiCall extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.eng */ @SerialVersionUID(0L) final case class Chain( - nexts: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Chain] { + nexts: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlRequest.NonEmpty with scalapb.lenses.Updatable[Chain] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -4788,9 +4788,9 @@ final case class Chain( }; } def clearNexts = copy(nexts = _root_.scala.Seq.empty) - def addNexts(__vs: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity *): Chain = addAllNexts(__vs) - def addAllNexts(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]): Chain = copy(nexts = nexts ++ __vs) - def withNexts(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]): Chain = copy(nexts = __v) + def addNexts(__vs: edu.uci.ics.amber.engine.common.ActorVirtualIdentity *): Chain = addAllNexts(__vs) + def addAllNexts(__vs: Iterable[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]): Chain = copy(nexts = nexts ++ __vs) + def withNexts(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]): Chain = copy(nexts = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => nexts @@ -4803,57 +4803,57 @@ final case class Chain( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain + def companion: edu.uci.ics.amber.engine.architecture.rpc.Chain.type = edu.uci.ics.amber.engine.architecture.rpc.Chain // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.Chain]) } -object Chain extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain = { - val __nexts: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] +object Chain extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Chain] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Chain] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.Chain = { + val __nexts: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __nexts += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__) + __nexts += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain( + edu.uci.ics.amber.engine.architecture.rpc.Chain( nexts = __nexts.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.Chain] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain( - nexts = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]]).getOrElse(_root_.scala.Seq.empty) + edu.uci.ics.amber.engine.architecture.rpc.Chain( + nexts = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(25) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(25) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(25) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(25) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.Chain( nexts = _root_.scala.Seq.empty ) - implicit class ChainLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain](_l) { - def nexts: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]] = field(_.nexts)((c_, f_) => c_.copy(nexts = f_)) + implicit class ChainLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Chain]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.Chain](_l) { + def nexts: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]] = field(_.nexts)((c_, f_) => c_.copy(nexts = f_)) } final val NEXTS_FIELD_NUMBER = 1 def of( - nexts: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain( + nexts: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.Chain = _root_.edu.uci.ics.amber.engine.architecture.rpc.Chain( nexts ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.Chain]) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ControlReturn.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlReturn.scala similarity index 57% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ControlReturn.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlReturn.scala index 08a17d51441..f220f8040a7 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ControlReturn.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlReturn.scala @@ -3,52 +3,52 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlreturns +package edu.uci.ics.amber.engine.architecture.rpc sealed trait ControlReturn extends scalapb.GeneratedSealedOneof { - type MessageType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage - final def isEmpty = this.isInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.Empty.type] + type MessageType = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage + final def isEmpty = this.isInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.Empty.type] final def isDefined = !isEmpty - final def asMessage: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.ControlReturnTypeMapper.toBase(this) - final def asNonEmpty: Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty] = if (isEmpty) None else Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty]) + final def asMessage: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage = edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.ControlReturnTypeMapper.toBase(this) + final def asNonEmpty: Option[edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty] = if (isEmpty) None else Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty]) } object ControlReturn { - case object Empty extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn + case object Empty extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturn - sealed trait NonEmpty extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn - def defaultInstance: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn = Empty + sealed trait NonEmpty extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturn + def defaultInstance: edu.uci.ics.amber.engine.architecture.rpc.ControlReturn = Empty - implicit val ControlReturnTypeMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn] = new _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn] { - override def toCustom(__base: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn = __base.sealedValue match { - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.RetrieveWorkflowStateResponse => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.PropagateChannelMarkerResponse => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.TakeGlobalCheckpointResponse => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.EvaluatePythonExpressionResponse => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.StartWorkflowResponse => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.WorkerStateResponse => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.WorkerMetricsResponse => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.FinalizeCheckpointResponse => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.Error => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.EmptyReturn => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.StringResponse => __v.value - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.IntResponse => __v.value - case edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.Empty => Empty - } - override def toBase(__custom: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage(__custom match { - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.RetrieveWorkflowStateResponse(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.PropagateChannelMarkerResponse(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.TakeGlobalCheckpointResponse(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.EvaluatePythonExpressionResponse(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.StartWorkflowResponse(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.WorkerStateResponse(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.WorkerMetricsResponse(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.FinalizeCheckpointResponse(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.Error(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.EmptyReturn(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.StringResponse(__v) - case __v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.IntResponse(__v) - case Empty => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.Empty + implicit val ControlReturnTypeMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn] = new _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn] { + override def toCustom(__base: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage): edu.uci.ics.amber.engine.architecture.rpc.ControlReturn = __base.sealedValue match { + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.RetrieveWorkflowStateResponse => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.PropagateChannelMarkerResponse => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.TakeGlobalCheckpointResponse => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.EvaluatePythonExpressionResponse => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.StartWorkflowResponse => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.WorkerStateResponse => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.WorkerMetricsResponse => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.FinalizeCheckpointResponse => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.Error => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.EmptyReturn => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.StringResponse => __v.value + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.IntResponse => __v.value + case edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.Empty => Empty + } + override def toBase(__custom: edu.uci.ics.amber.engine.architecture.rpc.ControlReturn): edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage(__custom match { + case __v: edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.RetrieveWorkflowStateResponse(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.PropagateChannelMarkerResponse(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.TakeGlobalCheckpointResponse(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.EvaluatePythonExpressionResponse(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.StartWorkflowResponse(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.WorkerStateResponse(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.WorkerMetricsResponse(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.FinalizeCheckpointResponse(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.ControlException => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.Error(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.EmptyReturn(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.StringResponse => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.StringResponse(__v) + case __v: edu.uci.ics.amber.engine.architecture.rpc.IntResponse => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.IntResponse(__v) + case Empty => edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.Empty }) } } @@ -56,7 +56,7 @@ object ControlReturn { */ @SerialVersionUID(0L) final case class ControlReturnMessage( - sealedValue: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue + sealedValue: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ControlReturnMessage] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -195,32 +195,32 @@ final case class ControlReturnMessage( __m.writeTo(_output__) }; } - def getRetrieveWorkflowStateResponse: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse = sealedValue.retrieveWorkflowStateResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.defaultInstance) - def withRetrieveWorkflowStateResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.RetrieveWorkflowStateResponse(__v)) - def getPropagateChannelMarkerResponse: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse = sealedValue.propagateChannelMarkerResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.defaultInstance) - def withPropagateChannelMarkerResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.PropagateChannelMarkerResponse(__v)) - def getTakeGlobalCheckpointResponse: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse = sealedValue.takeGlobalCheckpointResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse.defaultInstance) - def withTakeGlobalCheckpointResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.TakeGlobalCheckpointResponse(__v)) - def getEvaluatePythonExpressionResponse: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse = sealedValue.evaluatePythonExpressionResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse.defaultInstance) - def withEvaluatePythonExpressionResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.EvaluatePythonExpressionResponse(__v)) - def getStartWorkflowResponse: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse = sealedValue.startWorkflowResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse.defaultInstance) - def withStartWorkflowResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.StartWorkflowResponse(__v)) - def getWorkerStateResponse: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse = sealedValue.workerStateResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse.defaultInstance) - def withWorkerStateResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.WorkerStateResponse(__v)) - def getWorkerMetricsResponse: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse = sealedValue.workerMetricsResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse.defaultInstance) - def withWorkerMetricsResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.WorkerMetricsResponse(__v)) - def getFinalizeCheckpointResponse: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse = sealedValue.finalizeCheckpointResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse.defaultInstance) - def withFinalizeCheckpointResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.FinalizeCheckpointResponse(__v)) - def getError: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError = sealedValue.error.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError.defaultInstance) - def withError(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.Error(__v)) - def getEmptyReturn: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = sealedValue.emptyReturn.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn.defaultInstance) - def withEmptyReturn(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.EmptyReturn(__v)) - def getStringResponse: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse = sealedValue.stringResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse.defaultInstance) - def withStringResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.StringResponse(__v)) - def getIntResponse: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse = sealedValue.intResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse.defaultInstance) - def withIntResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.IntResponse(__v)) - def clearSealedValue: ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.Empty) - def withSealedValue(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue): ControlReturnMessage = copy(sealedValue = __v) + def getRetrieveWorkflowStateResponse: edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse = sealedValue.retrieveWorkflowStateResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.defaultInstance) + def withRetrieveWorkflowStateResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.RetrieveWorkflowStateResponse(__v)) + def getPropagateChannelMarkerResponse: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse = sealedValue.propagateChannelMarkerResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.defaultInstance) + def withPropagateChannelMarkerResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.PropagateChannelMarkerResponse(__v)) + def getTakeGlobalCheckpointResponse: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse = sealedValue.takeGlobalCheckpointResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse.defaultInstance) + def withTakeGlobalCheckpointResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.TakeGlobalCheckpointResponse(__v)) + def getEvaluatePythonExpressionResponse: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse = sealedValue.evaluatePythonExpressionResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse.defaultInstance) + def withEvaluatePythonExpressionResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.EvaluatePythonExpressionResponse(__v)) + def getStartWorkflowResponse: edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse = sealedValue.startWorkflowResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse.defaultInstance) + def withStartWorkflowResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.StartWorkflowResponse(__v)) + def getWorkerStateResponse: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse = sealedValue.workerStateResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse.defaultInstance) + def withWorkerStateResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.WorkerStateResponse(__v)) + def getWorkerMetricsResponse: edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse = sealedValue.workerMetricsResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse.defaultInstance) + def withWorkerMetricsResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.WorkerMetricsResponse(__v)) + def getFinalizeCheckpointResponse: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse = sealedValue.finalizeCheckpointResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse.defaultInstance) + def withFinalizeCheckpointResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.FinalizeCheckpointResponse(__v)) + def getError: edu.uci.ics.amber.engine.architecture.rpc.ControlException = sealedValue.error.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlException.defaultInstance) + def withError(__v: edu.uci.ics.amber.engine.architecture.rpc.ControlException): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.Error(__v)) + def getEmptyReturn: edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = sealedValue.emptyReturn.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn.defaultInstance) + def withEmptyReturn(__v: edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.EmptyReturn(__v)) + def getStringResponse: edu.uci.ics.amber.engine.architecture.rpc.StringResponse = sealedValue.stringResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.StringResponse.defaultInstance) + def withStringResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.StringResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.StringResponse(__v)) + def getIntResponse: edu.uci.ics.amber.engine.architecture.rpc.IntResponse = sealedValue.intResponse.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.IntResponse.defaultInstance) + def withIntResponse(__v: edu.uci.ics.amber.engine.architecture.rpc.IntResponse): ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.IntResponse(__v)) + def clearSealedValue: ControlReturnMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.Empty) + def withSealedValue(__v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue): ControlReturnMessage = copy(sealedValue = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => sealedValue.retrieveWorkflowStateResponse.orNull @@ -255,95 +255,95 @@ final case class ControlReturnMessage( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage - def toControlReturn: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.ControlReturnTypeMapper.toCustom(this) + def companion: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.type = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage + def toControlReturn: edu.uci.ics.amber.engine.architecture.rpc.ControlReturn = edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.ControlReturnTypeMapper.toCustom(this) // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.ControlReturn]) } -object ControlReturnMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage = { - var __sealedValue: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.Empty +object ControlReturnMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage = { + var __sealedValue: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.Empty var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.RetrieveWorkflowStateResponse(__sealedValue.retrieveWorkflowStateResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.RetrieveWorkflowStateResponse(__sealedValue.retrieveWorkflowStateResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.PropagateChannelMarkerResponse(__sealedValue.propagateChannelMarkerResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.PropagateChannelMarkerResponse(__sealedValue.propagateChannelMarkerResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 26 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.TakeGlobalCheckpointResponse(__sealedValue.takeGlobalCheckpointResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.TakeGlobalCheckpointResponse(__sealedValue.takeGlobalCheckpointResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 34 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.EvaluatePythonExpressionResponse(__sealedValue.evaluatePythonExpressionResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.EvaluatePythonExpressionResponse(__sealedValue.evaluatePythonExpressionResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 42 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.StartWorkflowResponse(__sealedValue.startWorkflowResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.StartWorkflowResponse(__sealedValue.startWorkflowResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 402 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.WorkerStateResponse(__sealedValue.workerStateResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.WorkerStateResponse(__sealedValue.workerStateResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 410 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.WorkerMetricsResponse(__sealedValue.workerMetricsResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.WorkerMetricsResponse(__sealedValue.workerMetricsResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 418 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.FinalizeCheckpointResponse(__sealedValue.finalizeCheckpointResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.FinalizeCheckpointResponse(__sealedValue.finalizeCheckpointResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 810 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.Error(__sealedValue.error.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.Error(__sealedValue.error.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ControlException](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 818 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.EmptyReturn(__sealedValue.emptyReturn.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.EmptyReturn(__sealedValue.emptyReturn.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 826 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.StringResponse(__sealedValue.stringResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.StringResponse(__sealedValue.stringResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.StringResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 834 => - __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.IntResponse(__sealedValue.intResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.IntResponse(__sealedValue.intResponse.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.IntResponse](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage( + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage( sealedValue = __sealedValue ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage( - sealedValue = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.RetrieveWorkflowStateResponse(_)) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.PropagateChannelMarkerResponse(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.TakeGlobalCheckpointResponse(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.EvaluatePythonExpressionResponse(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(5).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.StartWorkflowResponse(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(50).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.WorkerStateResponse(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(51).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.WorkerMetricsResponse(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(52).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.FinalizeCheckpointResponse(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(101).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.Error(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(102).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.EmptyReturn(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(103).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.StringResponse(_))) - .orElse[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(104).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.IntResponse(_))) - .getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.Empty) + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage( + sealedValue = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.RetrieveWorkflowStateResponse(_)) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.PropagateChannelMarkerResponse(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.TakeGlobalCheckpointResponse(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.EvaluatePythonExpressionResponse(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(5).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.StartWorkflowResponse(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(50).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.WorkerStateResponse(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(51).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.WorkerMetricsResponse(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(52).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.FinalizeCheckpointResponse(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(101).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlException]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.Error(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(102).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.EmptyReturn(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(103).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.StringResponse(_))) + .orElse[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(104).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.IntResponse]]).map(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.IntResponse(_))) + .getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.Empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse - case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse - case 3 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse - case 4 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse - case 5 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse - case 50 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse - case 51 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse - case 52 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse - case 101 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError - case 102 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - case 103 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse - case 104 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse + case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse + case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse + case 3 => __out = edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse + case 4 => __out = edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse + case 5 => __out = edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse + case 50 => __out = edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse + case 51 => __out = edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse + case 52 => __out = edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse + case 101 => __out = edu.uci.ics.amber.engine.architecture.rpc.ControlException + case 102 => __out = edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + case 103 => __out = edu.uci.ics.amber.engine.architecture.rpc.StringResponse + case 104 => __out = edu.uci.ics.amber.engine.architecture.rpc.IntResponse } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage( - sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.Empty + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage( + sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.Empty ) sealed trait SealedValue extends _root_.scalapb.GeneratedOneof { def isEmpty: _root_.scala.Boolean = false @@ -360,22 +360,22 @@ object ControlReturnMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ic def isEmptyReturn: _root_.scala.Boolean = false def isStringResponse: _root_.scala.Boolean = false def isIntResponse: _root_.scala.Boolean = false - def retrieveWorkflowStateResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse] = _root_.scala.None - def propagateChannelMarkerResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse] = _root_.scala.None - def takeGlobalCheckpointResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse] = _root_.scala.None - def evaluatePythonExpressionResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse] = _root_.scala.None - def startWorkflowResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse] = _root_.scala.None - def workerStateResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = _root_.scala.None - def workerMetricsResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse] = _root_.scala.None - def finalizeCheckpointResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse] = _root_.scala.None - def error: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError] = _root_.scala.None - def emptyReturn: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = _root_.scala.None - def stringResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = _root_.scala.None - def intResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = _root_.scala.None + def retrieveWorkflowStateResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse] = _root_.scala.None + def propagateChannelMarkerResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse] = _root_.scala.None + def takeGlobalCheckpointResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse] = _root_.scala.None + def evaluatePythonExpressionResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse] = _root_.scala.None + def startWorkflowResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse] = _root_.scala.None + def workerStateResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = _root_.scala.None + def workerMetricsResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse] = _root_.scala.None + def finalizeCheckpointResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse] = _root_.scala.None + def error: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlException] = _root_.scala.None + def emptyReturn: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.scala.None + def stringResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = _root_.scala.None + def intResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = _root_.scala.None } object SealedValue { @SerialVersionUID(0L) - case object Empty extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { + case object Empty extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { type ValueType = _root_.scala.Nothing override def isEmpty: _root_.scala.Boolean = true override def isDefined: _root_.scala.Boolean = false @@ -384,120 +384,120 @@ object ControlReturnMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ic } @SerialVersionUID(0L) - final case class RetrieveWorkflowStateResponse(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse) extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse + final case class RetrieveWorkflowStateResponse(value: edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse) extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse override def isRetrieveWorkflowStateResponse: _root_.scala.Boolean = true - override def retrieveWorkflowStateResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse] = Some(value) + override def retrieveWorkflowStateResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse] = Some(value) override def number: _root_.scala.Int = 1 } @SerialVersionUID(0L) - final case class PropagateChannelMarkerResponse(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse) extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse + final case class PropagateChannelMarkerResponse(value: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse) extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse override def isPropagateChannelMarkerResponse: _root_.scala.Boolean = true - override def propagateChannelMarkerResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse] = Some(value) + override def propagateChannelMarkerResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse] = Some(value) override def number: _root_.scala.Int = 2 } @SerialVersionUID(0L) - final case class TakeGlobalCheckpointResponse(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse) extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse + final case class TakeGlobalCheckpointResponse(value: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse) extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse override def isTakeGlobalCheckpointResponse: _root_.scala.Boolean = true - override def takeGlobalCheckpointResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse] = Some(value) + override def takeGlobalCheckpointResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse] = Some(value) override def number: _root_.scala.Int = 3 } @SerialVersionUID(0L) - final case class EvaluatePythonExpressionResponse(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse) extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse + final case class EvaluatePythonExpressionResponse(value: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse) extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse override def isEvaluatePythonExpressionResponse: _root_.scala.Boolean = true - override def evaluatePythonExpressionResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse] = Some(value) + override def evaluatePythonExpressionResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse] = Some(value) override def number: _root_.scala.Int = 4 } @SerialVersionUID(0L) - final case class StartWorkflowResponse(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse) extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse + final case class StartWorkflowResponse(value: edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse) extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse override def isStartWorkflowResponse: _root_.scala.Boolean = true - override def startWorkflowResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse] = Some(value) + override def startWorkflowResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse] = Some(value) override def number: _root_.scala.Int = 5 } @SerialVersionUID(0L) - final case class WorkerStateResponse(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse) extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse + final case class WorkerStateResponse(value: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse) extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse override def isWorkerStateResponse: _root_.scala.Boolean = true - override def workerStateResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = Some(value) + override def workerStateResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = Some(value) override def number: _root_.scala.Int = 50 } @SerialVersionUID(0L) - final case class WorkerMetricsResponse(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse) extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse + final case class WorkerMetricsResponse(value: edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse) extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse override def isWorkerMetricsResponse: _root_.scala.Boolean = true - override def workerMetricsResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse] = Some(value) + override def workerMetricsResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse] = Some(value) override def number: _root_.scala.Int = 51 } @SerialVersionUID(0L) - final case class FinalizeCheckpointResponse(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse) extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse + final case class FinalizeCheckpointResponse(value: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse) extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse override def isFinalizeCheckpointResponse: _root_.scala.Boolean = true - override def finalizeCheckpointResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse] = Some(value) + override def finalizeCheckpointResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse] = Some(value) override def number: _root_.scala.Int = 52 } @SerialVersionUID(0L) - final case class Error(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError) extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError + final case class Error(value: edu.uci.ics.amber.engine.architecture.rpc.ControlException) extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.ControlException override def isError: _root_.scala.Boolean = true - override def error: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError] = Some(value) + override def error: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlException] = Some(value) override def number: _root_.scala.Int = 101 } @SerialVersionUID(0L) - final case class EmptyReturn(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn) extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn + final case class EmptyReturn(value: edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn) extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn override def isEmptyReturn: _root_.scala.Boolean = true - override def emptyReturn: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = Some(value) + override def emptyReturn: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = Some(value) override def number: _root_.scala.Int = 102 } @SerialVersionUID(0L) - final case class StringResponse(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse) extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse + final case class StringResponse(value: edu.uci.ics.amber.engine.architecture.rpc.StringResponse) extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.StringResponse override def isStringResponse: _root_.scala.Boolean = true - override def stringResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = Some(value) + override def stringResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = Some(value) override def number: _root_.scala.Int = 103 } @SerialVersionUID(0L) - final case class IntResponse(value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse) extends edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse + final case class IntResponse(value: edu.uci.ics.amber.engine.architecture.rpc.IntResponse) extends edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.rpc.IntResponse override def isIntResponse: _root_.scala.Boolean = true - override def intResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = Some(value) + override def intResponse: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = Some(value) override def number: _root_.scala.Int = 104 } } - implicit class ControlReturnMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage](_l) { - def retrieveWorkflowStateResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse] = field(_.getRetrieveWorkflowStateResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.RetrieveWorkflowStateResponse(f_))) - def propagateChannelMarkerResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse] = field(_.getPropagateChannelMarkerResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.PropagateChannelMarkerResponse(f_))) - def takeGlobalCheckpointResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse] = field(_.getTakeGlobalCheckpointResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.TakeGlobalCheckpointResponse(f_))) - def evaluatePythonExpressionResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse] = field(_.getEvaluatePythonExpressionResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.EvaluatePythonExpressionResponse(f_))) - def startWorkflowResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse] = field(_.getStartWorkflowResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.StartWorkflowResponse(f_))) - def workerStateResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = field(_.getWorkerStateResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.WorkerStateResponse(f_))) - def workerMetricsResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse] = field(_.getWorkerMetricsResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.WorkerMetricsResponse(f_))) - def finalizeCheckpointResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse] = field(_.getFinalizeCheckpointResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.FinalizeCheckpointResponse(f_))) - def error: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError] = field(_.getError)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.Error(f_))) - def emptyReturn: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = field(_.getEmptyReturn)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.EmptyReturn(f_))) - def stringResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = field(_.getStringResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.StringResponse(f_))) - def intResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = field(_.getIntResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue.IntResponse(f_))) - def sealedValue: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue] = field(_.sealedValue)((c_, f_) => c_.copy(sealedValue = f_)) + implicit class ControlReturnMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage](_l) { + def retrieveWorkflowStateResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse] = field(_.getRetrieveWorkflowStateResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.RetrieveWorkflowStateResponse(f_))) + def propagateChannelMarkerResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse] = field(_.getPropagateChannelMarkerResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.PropagateChannelMarkerResponse(f_))) + def takeGlobalCheckpointResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse] = field(_.getTakeGlobalCheckpointResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.TakeGlobalCheckpointResponse(f_))) + def evaluatePythonExpressionResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse] = field(_.getEvaluatePythonExpressionResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.EvaluatePythonExpressionResponse(f_))) + def startWorkflowResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse] = field(_.getStartWorkflowResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.StartWorkflowResponse(f_))) + def workerStateResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = field(_.getWorkerStateResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.WorkerStateResponse(f_))) + def workerMetricsResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse] = field(_.getWorkerMetricsResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.WorkerMetricsResponse(f_))) + def finalizeCheckpointResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse] = field(_.getFinalizeCheckpointResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.FinalizeCheckpointResponse(f_))) + def error: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlException] = field(_.getError)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.Error(f_))) + def emptyReturn: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = field(_.getEmptyReturn)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.EmptyReturn(f_))) + def stringResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = field(_.getStringResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.StringResponse(f_))) + def intResponse: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = field(_.getIntResponse)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue.IntResponse(f_))) + def sealedValue: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue] = field(_.sealedValue)((c_, f_) => c_.copy(sealedValue = f_)) } - final val RETRIEVEWORKFLOWSTATERESPONSE_FIELD_NUMBER = 1 - final val PROPAGATECHANNELMARKERRESPONSE_FIELD_NUMBER = 2 - final val TAKEGLOBALCHECKPOINTRESPONSE_FIELD_NUMBER = 3 - final val EVALUATEPYTHONEXPRESSIONRESPONSE_FIELD_NUMBER = 4 - final val STARTWORKFLOWRESPONSE_FIELD_NUMBER = 5 - final val WORKERSTATERESPONSE_FIELD_NUMBER = 50 - final val WORKERMETRICSRESPONSE_FIELD_NUMBER = 51 - final val FINALIZECHECKPOINTRESPONSE_FIELD_NUMBER = 52 + final val RETRIEVE_WORKFLOW_STATE_RESPONSE_FIELD_NUMBER = 1 + final val PROPAGATE_CHANNEL_MARKER_RESPONSE_FIELD_NUMBER = 2 + final val TAKE_GLOBAL_CHECKPOINT_RESPONSE_FIELD_NUMBER = 3 + final val EVALUATE_PYTHON_EXPRESSION_RESPONSE_FIELD_NUMBER = 4 + final val START_WORKFLOW_RESPONSE_FIELD_NUMBER = 5 + final val WORKER_STATE_RESPONSE_FIELD_NUMBER = 50 + final val WORKER_METRICS_RESPONSE_FIELD_NUMBER = 51 + final val FINALIZE_CHECKPOINT_RESPONSE_FIELD_NUMBER = 52 final val ERROR_FIELD_NUMBER = 101 - final val EMPTYRETURN_FIELD_NUMBER = 102 - final val STRINGRESPONSE_FIELD_NUMBER = 103 - final val INTRESPONSE_FIELD_NUMBER = 104 + final val EMPTY_RETURN_FIELD_NUMBER = 102 + final val STRING_RESPONSE_FIELD_NUMBER = 103 + final val INT_RESPONSE_FIELD_NUMBER = 104 def of( - sealedValue: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.SealedValue - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage( + sealedValue: edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.SealedValue + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage = _root_.edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage( sealedValue ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ControlReturn]) @@ -506,13 +506,13 @@ object ControlReturnMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ic @SerialVersionUID(0L) final case class RetrieveWorkflowStateResponse( state: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, _root_.scala.Predef.String] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty with scalapb.lenses.Updatable[RetrieveWorkflowStateResponse] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty with scalapb.lenses.Updatable[RetrieveWorkflowStateResponse] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { var __size = 0 state.foreach { __item => - val __value = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse._typemapper_state.toBase(__item) + val __value = edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse._typemapper_state.toBase(__item) __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } __size @@ -528,7 +528,7 @@ final case class RetrieveWorkflowStateResponse( } def writeTo(`_output__`: _root_.com.google.protobuf.CodedOutputStream): _root_.scala.Unit = { state.foreach { __v => - val __m = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse._typemapper_state.toBase(__v) + val __m = edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse._typemapper_state.toBase(__v) _output__.writeTag(1, 2) _output__.writeUInt32NoTag(__m.serializedSize) __m.writeTo(_output__) @@ -540,23 +540,23 @@ final case class RetrieveWorkflowStateResponse( def withState(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, _root_.scala.Predef.String]): RetrieveWorkflowStateResponse = copy(state = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { - case 1 => state.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse._typemapper_state.toBase(_)).toSeq + case 1 => state.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse._typemapper_state.toBase(_)).toSeq } } def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = { _root_.scala.Predef.require(__field.containingMessage eq companion.scalaDescriptor) (__field.number: @_root_.scala.unchecked) match { - case 1 => _root_.scalapb.descriptors.PRepeated(state.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse._typemapper_state.toBase(_).toPMessage).toVector) + case 1 => _root_.scalapb.descriptors.PRepeated(state.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse._typemapper_state.toBase(_).toPMessage).toVector) } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse + def companion: edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.type = edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse]) } -object RetrieveWorkflowStateResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse = { +object RetrieveWorkflowStateResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse = { val __state: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, _root_.scala.Predef.String), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, _root_.scala.Predef.String]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, _root_.scala.Predef.String] var _done__ = false while (!_done__) { @@ -564,37 +564,37 @@ object RetrieveWorkflowStateResponse extends scalapb.GeneratedMessageCompanion[e _tag__ match { case 0 => _done__ = true case 10 => - __state += edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse._typemapper_state.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry](_input__)) + __state += edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse._typemapper_state.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry](_input__)) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse( + edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse( state = __state.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse( - state = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse._typemapper_state.toCustom(_)).toMap + edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse( + state = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse._typemapper_state.toCustom(_)).toMap ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(6) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(6) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(6) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(6) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry + case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry + _root_.edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry ) def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse( state = _root_.scala.collection.immutable.Map.empty ) @SerialVersionUID(0L) @@ -667,13 +667,13 @@ object RetrieveWorkflowStateResponse extends scalapb.GeneratedMessageCompanion[e } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry + def companion: edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry.type = edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry]) } - object StateEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry = { + object StateEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry = { var __key: _root_.scala.Predef.String = "" var __value: _root_.scala.Predef.String = "" var _done__ = false @@ -688,57 +688,57 @@ object RetrieveWorkflowStateResponse extends scalapb.GeneratedMessageCompanion[e case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry( + edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry( key = __key, value = __value ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry( + edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry( key = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.javaDescriptor.getNestedTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.scalaDescriptor.nestedMessages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.javaDescriptor.getNestedTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.scalaDescriptor.nestedMessages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry( key = "", value = "" ) - implicit class StateEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry](_l) { + implicit class StateEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry](_l) { def key: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.key)((c_, f_) => c_.copy(key = f_)) def value: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.value)((c_, f_) => c_.copy(value = f_)) } final val KEY_FIELD_NUMBER = 1 final val VALUE_FIELD_NUMBER = 2 @transient - implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)] = - _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)](__m => (__m.key, __m.value))(__p => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry(__p._1, __p._2)) + implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)] = + _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)](__m => (__m.key, __m.value))(__p => edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry(__p._1, __p._2)) def of( key: _root_.scala.Predef.String, value: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry = _root_.edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry( key, value ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry]) } - implicit class RetrieveWorkflowStateResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse](_l) { + implicit class RetrieveWorkflowStateResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse](_l) { def state: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, _root_.scala.Predef.String]] = field(_.state)((c_, f_) => c_.copy(state = f_)) } final val STATE_FIELD_NUMBER = 1 @transient - private[controlreturns] val _typemapper_state: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse.StateEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)]] + private[rpc] val _typemapper_state: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse.StateEntry, (_root_.scala.Predef.String, _root_.scala.Predef.String)]] def of( state: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, _root_.scala.Predef.String] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse( state ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse]) @@ -746,14 +746,14 @@ object RetrieveWorkflowStateResponse extends scalapb.GeneratedMessageCompanion[e @SerialVersionUID(0L) final case class PropagateChannelMarkerResponse( - returns: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty with scalapb.lenses.Updatable[PropagateChannelMarkerResponse] { + returns: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn] + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty with scalapb.lenses.Updatable[PropagateChannelMarkerResponse] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { var __size = 0 returns.foreach { __item => - val __value = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse._typemapper_returns.toBase(__item) + val __value = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse._typemapper_returns.toBase(__item) __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } __size @@ -769,79 +769,79 @@ final case class PropagateChannelMarkerResponse( } def writeTo(`_output__`: _root_.com.google.protobuf.CodedOutputStream): _root_.scala.Unit = { returns.foreach { __v => - val __m = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse._typemapper_returns.toBase(__v) + val __m = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse._typemapper_returns.toBase(__v) _output__.writeTag(1, 2) _output__.writeUInt32NoTag(__m.serializedSize) __m.writeTo(_output__) }; } def clearReturns = copy(returns = _root_.scala.collection.immutable.Map.empty) - def addReturns(__vs: (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn) *): PropagateChannelMarkerResponse = addAllReturns(__vs) - def addAllReturns(__vs: Iterable[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn)]): PropagateChannelMarkerResponse = copy(returns = returns ++ __vs) - def withReturns(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn]): PropagateChannelMarkerResponse = copy(returns = __v) + def addReturns(__vs: (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn) *): PropagateChannelMarkerResponse = addAllReturns(__vs) + def addAllReturns(__vs: Iterable[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn)]): PropagateChannelMarkerResponse = copy(returns = returns ++ __vs) + def withReturns(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn]): PropagateChannelMarkerResponse = copy(returns = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { - case 1 => returns.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse._typemapper_returns.toBase(_)).toSeq + case 1 => returns.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse._typemapper_returns.toBase(_)).toSeq } } def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = { _root_.scala.Predef.require(__field.containingMessage eq companion.scalaDescriptor) (__field.number: @_root_.scala.unchecked) match { - case 1 => _root_.scalapb.descriptors.PRepeated(returns.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse._typemapper_returns.toBase(_).toPMessage).toVector) + case 1 => _root_.scalapb.descriptors.PRepeated(returns.iterator.map(edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse._typemapper_returns.toBase(_).toPMessage).toVector) } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse + def companion: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.type = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse]) } -object PropagateChannelMarkerResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse = { - val __returns: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn] +object PropagateChannelMarkerResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse = { + val __returns: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __returns += edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse._typemapper_returns.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry](_input__)) + __returns += edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse._typemapper_returns.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry](_input__)) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse( + edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse( returns = __returns.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse( - returns = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse._typemapper_returns.toCustom(_)).toMap + edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse( + returns = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse._typemapper_returns.toCustom(_)).toMap ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(8) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(8) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(8) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(8) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry + case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry + _root_.edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry ) def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse( returns = _root_.scala.collection.immutable.Map.empty ) @SerialVersionUID(0L) final case class ReturnsEntry( key: _root_.scala.Predef.String, - value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn + value: edu.uci.ics.amber.engine.architecture.rpc.ControlReturn ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ReturnsEntry] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -856,7 +856,7 @@ object PropagateChannelMarkerResponse extends scalapb.GeneratedMessageCompanion[ }; { - val __value = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toBase(value) + val __value = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toBase(value) if (__value.serializedSize != 0) { __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } @@ -880,7 +880,7 @@ object PropagateChannelMarkerResponse extends scalapb.GeneratedMessageCompanion[ } }; { - val __v = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toBase(value) + val __v = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toBase(value) if (__v.serializedSize != 0) { _output__.writeTag(2, 2) _output__.writeUInt32NoTag(__v.serializedSize) @@ -889,7 +889,7 @@ object PropagateChannelMarkerResponse extends scalapb.GeneratedMessageCompanion[ }; } def withKey(__v: _root_.scala.Predef.String): ReturnsEntry = copy(key = __v) - def withValue(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn): ReturnsEntry = copy(value = __v) + def withValue(__v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturn): ReturnsEntry = copy(value = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -897,8 +897,8 @@ object PropagateChannelMarkerResponse extends scalapb.GeneratedMessageCompanion[ if (__t != "") __t else null } case 2 => { - val __t = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toBase(value) - if (__t != edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.defaultInstance) __t else null + val __t = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toBase(value) + if (__t != edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.defaultInstance) __t else null } } } @@ -906,19 +906,19 @@ object PropagateChannelMarkerResponse extends scalapb.GeneratedMessageCompanion[ _root_.scala.Predef.require(__field.containingMessage eq companion.scalaDescriptor) (__field.number: @_root_.scala.unchecked) match { case 1 => _root_.scalapb.descriptors.PString(key) - case 2 => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toBase(value).toPMessage + case 2 => edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toBase(value).toPMessage } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry + def companion: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry.type = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry]) } - object ReturnsEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry = { + object ReturnsEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry = { var __key: _root_.scala.Predef.String = "" - var __value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage] = _root_.scala.None + var __value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -927,69 +927,69 @@ object PropagateChannelMarkerResponse extends scalapb.GeneratedMessageCompanion[ case 10 => __key = _input__.readStringRequireUtf8() case 18 => - __value = _root_.scala.Some(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __value = _root_.scala.Some(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry( + edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry( key = __key, - value = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toCustom(__value.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.defaultInstance)) + value = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toCustom(__value.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.defaultInstance)) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry( + edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry( key = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), - value = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toCustom(__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.defaultInstance)) + value = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toCustom(__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.defaultInstance)) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.javaDescriptor.getNestedTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.scalaDescriptor.nestedMessages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.javaDescriptor.getNestedTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.scalaDescriptor.nestedMessages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage + case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry( key = "", - value = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toCustom(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.defaultInstance) + value = edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry._typemapper_value.toCustom(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.defaultInstance) ) - implicit class ReturnsEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry](_l) { + implicit class ReturnsEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry](_l) { def key: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.key)((c_, f_) => c_.copy(key = f_)) - def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn] = field(_.value)((c_, f_) => c_.copy(value = f_)) + def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn] = field(_.value)((c_, f_) => c_.copy(value = f_)) } final val KEY_FIELD_NUMBER = 1 final val VALUE_FIELD_NUMBER = 2 @transient - private[controlreturns] val _typemapper_value: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn]] + private[rpc] val _typemapper_value: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn]] @transient - implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn)] = - _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn)](__m => (__m.key, __m.value))(__p => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry(__p._1, __p._2)) + implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn)] = + _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn)](__m => (__m.key, __m.value))(__p => edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry(__p._1, __p._2)) def of( key: _root_.scala.Predef.String, - value: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry( + value: edu.uci.ics.amber.engine.architecture.rpc.ControlReturn + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry = _root_.edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry( key, value ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry]) } - implicit class PropagateChannelMarkerResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse](_l) { - def returns: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn]] = field(_.returns)((c_, f_) => c_.copy(returns = f_)) + implicit class PropagateChannelMarkerResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse](_l) { + def returns: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn]] = field(_.returns)((c_, f_) => c_.copy(returns = f_)) } final val RETURNS_FIELD_NUMBER = 1 @transient - private[controlreturns] val _typemapper_returns: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse.ReturnsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn)]] + private[rpc] val _typemapper_returns: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse.ReturnsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn)]] def of( - returns: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse( + returns: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn] + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse( returns ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse]) @@ -998,7 +998,7 @@ object PropagateChannelMarkerResponse extends scalapb.GeneratedMessageCompanion[ @SerialVersionUID(0L) final case class TakeGlobalCheckpointResponse( totalSize: _root_.scala.Long - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty with scalapb.lenses.Updatable[TakeGlobalCheckpointResponse] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty with scalapb.lenses.Updatable[TakeGlobalCheckpointResponse] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1045,13 +1045,13 @@ final case class TakeGlobalCheckpointResponse( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse + def companion: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse.type = edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse]) } -object TakeGlobalCheckpointResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse = { +object TakeGlobalCheckpointResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse = { var __totalSize: _root_.scala.Long = 0L var _done__ = false while (!_done__) { @@ -1063,33 +1063,33 @@ object TakeGlobalCheckpointResponse extends scalapb.GeneratedMessageCompanion[ed case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse( + edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse( totalSize = __totalSize ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse( + edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse( totalSize = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Long]).getOrElse(0L) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(9) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(9) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(9) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(9) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse( totalSize = 0L ) - implicit class TakeGlobalCheckpointResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse](_l) { + implicit class TakeGlobalCheckpointResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse](_l) { def totalSize: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.totalSize)((c_, f_) => c_.copy(totalSize = f_)) } final val TOTALSIZE_FIELD_NUMBER = 1 def of( totalSize: _root_.scala.Long - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse( totalSize ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse]) @@ -1097,8 +1097,8 @@ object TakeGlobalCheckpointResponse extends scalapb.GeneratedMessageCompanion[ed @SerialVersionUID(0L) final case class EvaluatePythonExpressionResponse( - values: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty with scalapb.lenses.Updatable[EvaluatePythonExpressionResponse] { + values: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty with scalapb.lenses.Updatable[EvaluatePythonExpressionResponse] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1127,9 +1127,9 @@ final case class EvaluatePythonExpressionResponse( }; } def clearValues = copy(values = _root_.scala.Seq.empty) - def addValues(__vs: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue *): EvaluatePythonExpressionResponse = addAllValues(__vs) - def addAllValues(__vs: Iterable[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue]): EvaluatePythonExpressionResponse = copy(values = values ++ __vs) - def withValues(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue]): EvaluatePythonExpressionResponse = copy(values = __v) + def addValues(__vs: edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue *): EvaluatePythonExpressionResponse = addAllValues(__vs) + def addAllValues(__vs: Iterable[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]): EvaluatePythonExpressionResponse = copy(values = values ++ __vs) + def withValues(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]): EvaluatePythonExpressionResponse = copy(values = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => values @@ -1142,57 +1142,57 @@ final case class EvaluatePythonExpressionResponse( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse + def companion: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse.type = edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse]) } -object EvaluatePythonExpressionResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse = { - val __values: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] +object EvaluatePythonExpressionResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse = { + val __values: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __values += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue](_input__) + __values += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse( + edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse( values = __values.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse( - values = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue]]).getOrElse(_root_.scala.Seq.empty) + edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse( + values = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(12) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(12) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(12) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(12) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue + case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse( values = _root_.scala.Seq.empty ) - implicit class EvaluatePythonExpressionResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse](_l) { - def values: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue]] = field(_.values)((c_, f_) => c_.copy(values = f_)) + implicit class EvaluatePythonExpressionResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse](_l) { + def values: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]] = field(_.values)((c_, f_) => c_.copy(values = f_)) } final val VALUES_FIELD_NUMBER = 1 def of( - values: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse( + values: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse( values ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse]) @@ -1200,8 +1200,8 @@ object EvaluatePythonExpressionResponse extends scalapb.GeneratedMessageCompanio @SerialVersionUID(0L) final case class StartWorkflowResponse( - workflowState: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty with scalapb.lenses.Updatable[StartWorkflowResponse] { + workflowState: edu.uci.ics.amber.engine.common.WorkflowAggregatedState + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty with scalapb.lenses.Updatable[StartWorkflowResponse] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1232,7 +1232,7 @@ final case class StartWorkflowResponse( } }; } - def withWorkflowState(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState): StartWorkflowResponse = copy(workflowState = __v) + def withWorkflowState(__v: edu.uci.ics.amber.engine.common.WorkflowAggregatedState): StartWorkflowResponse = copy(workflowState = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -1248,55 +1248,55 @@ final case class StartWorkflowResponse( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse + def companion: edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse.type = edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse]) } -object StartWorkflowResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse = { - var __workflowState: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.UNINITIALIZED +object StartWorkflowResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse = { + var __workflowState: edu.uci.ics.amber.engine.common.WorkflowAggregatedState = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.UNINITIALIZED var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 8 => - __workflowState = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.fromValue(_input__.readEnum()) + __workflowState = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.fromValue(_input__.readEnum()) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse( + edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse( workflowState = __workflowState ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse( - workflowState = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.UNINITIALIZED.scalaValueDescriptor).number) + edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse( + workflowState = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.common.WorkflowAggregatedState.UNINITIALIZED.scalaValueDescriptor).number) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(13) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(13) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(13) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(13) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = { (__fieldNumber: @_root_.scala.unchecked) match { - case 1 => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState + case 1 => edu.uci.ics.amber.engine.common.WorkflowAggregatedState } } - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse( - workflowState = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.UNINITIALIZED + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse( + workflowState = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.UNINITIALIZED ) - implicit class StartWorkflowResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse](_l) { - def workflowState: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState] = field(_.workflowState)((c_, f_) => c_.copy(workflowState = f_)) + implicit class StartWorkflowResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse](_l) { + def workflowState: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.WorkflowAggregatedState] = field(_.workflowState)((c_, f_) => c_.copy(workflowState = f_)) } - final val WORKFLOWSTATE_FIELD_NUMBER = 1 + final val WORKFLOW_STATE_FIELD_NUMBER = 1 def of( - workflowState: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse( + workflowState: edu.uci.ics.amber.engine.common.WorkflowAggregatedState + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse( workflowState ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse]) @@ -1304,8 +1304,8 @@ object StartWorkflowResponse extends scalapb.GeneratedMessageCompanion[edu.uci.i @SerialVersionUID(0L) final case class WorkerStateResponse( - state: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty with scalapb.lenses.Updatable[WorkerStateResponse] { + state: edu.uci.ics.amber.engine.architecture.worker.WorkerState + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty with scalapb.lenses.Updatable[WorkerStateResponse] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1336,7 +1336,7 @@ final case class WorkerStateResponse( } }; } - def withState(__v: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState): WorkerStateResponse = copy(state = __v) + def withState(__v: edu.uci.ics.amber.engine.architecture.worker.WorkerState): WorkerStateResponse = copy(state = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -1352,55 +1352,55 @@ final case class WorkerStateResponse( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse + def companion: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse.type = edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]) } -object WorkerStateResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse = { - var __state: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.UNINITIALIZED +object WorkerStateResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse = { + var __state: edu.uci.ics.amber.engine.architecture.worker.WorkerState = edu.uci.ics.amber.engine.architecture.worker.WorkerState.UNINITIALIZED var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 8 => - __state = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.fromValue(_input__.readEnum()) + __state = edu.uci.ics.amber.engine.architecture.worker.WorkerState.fromValue(_input__.readEnum()) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse( + edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse( state = __state ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse( - state = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.UNINITIALIZED.scalaValueDescriptor).number) + edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse( + state = edu.uci.ics.amber.engine.architecture.worker.WorkerState.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.worker.WorkerState.UNINITIALIZED.scalaValueDescriptor).number) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(14) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(14) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(14) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(14) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = { (__fieldNumber: @_root_.scala.unchecked) match { - case 1 => edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState + case 1 => edu.uci.ics.amber.engine.architecture.worker.WorkerState } } - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse( - state = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.UNINITIALIZED + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse( + state = edu.uci.ics.amber.engine.architecture.worker.WorkerState.UNINITIALIZED ) - implicit class WorkerStateResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse](_l) { - def state: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState] = field(_.state)((c_, f_) => c_.copy(state = f_)) + implicit class WorkerStateResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse](_l) { + def state: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.WorkerState] = field(_.state)((c_, f_) => c_.copy(state = f_)) } final val STATE_FIELD_NUMBER = 1 def of( - state: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse( + state: edu.uci.ics.amber.engine.architecture.worker.WorkerState + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse( state ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]) @@ -1408,8 +1408,8 @@ object WorkerStateResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics @SerialVersionUID(0L) final case class WorkerMetricsResponse( - metrics: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty with scalapb.lenses.Updatable[WorkerMetricsResponse] { + metrics: edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty with scalapb.lenses.Updatable[WorkerMetricsResponse] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1442,12 +1442,12 @@ final case class WorkerMetricsResponse( } }; } - def withMetrics(__v: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics): WorkerMetricsResponse = copy(metrics = __v) + def withMetrics(__v: edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics): WorkerMetricsResponse = copy(metrics = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = metrics - if (__t != edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics.defaultInstance) __t else null } } } @@ -1458,57 +1458,57 @@ final case class WorkerMetricsResponse( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse + def companion: edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse.type = edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse]) } -object WorkerMetricsResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse = { - var __metrics: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics] = _root_.scala.None +object WorkerMetricsResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse = { + var __metrics: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __metrics = _root_.scala.Some(__metrics.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __metrics = _root_.scala.Some(__metrics.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse( - metrics = __metrics.getOrElse(edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics.defaultInstance) + edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse( + metrics = __metrics.getOrElse(edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics.defaultInstance) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse( - metrics = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics]).getOrElse(edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics.defaultInstance) + edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse( + metrics = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics]).getOrElse(edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics.defaultInstance) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(15) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(15) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(15) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(15) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics + case 1 => __out = edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse( - metrics = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics.defaultInstance + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse( + metrics = edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics.defaultInstance ) - implicit class WorkerMetricsResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse](_l) { - def metrics: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics] = field(_.metrics)((c_, f_) => c_.copy(metrics = f_)) + implicit class WorkerMetricsResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse](_l) { + def metrics: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics] = field(_.metrics)((c_, f_) => c_.copy(metrics = f_)) } final val METRICS_FIELD_NUMBER = 1 def of( - metrics: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse( + metrics: edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse( metrics ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse]) @@ -1517,7 +1517,7 @@ object WorkerMetricsResponse extends scalapb.GeneratedMessageCompanion[edu.uci.i @SerialVersionUID(0L) final case class FinalizeCheckpointResponse( size: _root_.scala.Long - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty with scalapb.lenses.Updatable[FinalizeCheckpointResponse] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty with scalapb.lenses.Updatable[FinalizeCheckpointResponse] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1564,13 +1564,13 @@ final case class FinalizeCheckpointResponse( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse + def companion: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse.type = edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse]) } -object FinalizeCheckpointResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse = { +object FinalizeCheckpointResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse = { var __size: _root_.scala.Long = 0L var _done__ = false while (!_done__) { @@ -1582,45 +1582,45 @@ object FinalizeCheckpointResponse extends scalapb.GeneratedMessageCompanion[edu. case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse( + edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse( size = __size ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse( + edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse( size = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Long]).getOrElse(0L) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(7) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(7) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(7) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(7) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse( size = 0L ) - implicit class FinalizeCheckpointResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse](_l) { + implicit class FinalizeCheckpointResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse](_l) { def size: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.size)((c_, f_) => c_.copy(size = f_)) } final val SIZE_FIELD_NUMBER = 1 def of( size: _root_.scala.Long - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse( size ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse]) } @SerialVersionUID(0L) -final case class ControlError( +final case class ControlException( errorMessage: _root_.scala.Predef.String, errorDetails: _root_.scala.Predef.String, stackTrace: _root_.scala.Predef.String, - language: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty with scalapb.lenses.Updatable[ControlError] { + language: edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty with scalapb.lenses.Updatable[ControlException] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1690,10 +1690,10 @@ final case class ControlError( } }; } - def withErrorMessage(__v: _root_.scala.Predef.String): ControlError = copy(errorMessage = __v) - def withErrorDetails(__v: _root_.scala.Predef.String): ControlError = copy(errorDetails = __v) - def withStackTrace(__v: _root_.scala.Predef.String): ControlError = copy(stackTrace = __v) - def withLanguage(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage): ControlError = copy(language = __v) + def withErrorMessage(__v: _root_.scala.Predef.String): ControlException = copy(errorMessage = __v) + def withErrorDetails(__v: _root_.scala.Predef.String): ControlException = copy(errorDetails = __v) + def withStackTrace(__v: _root_.scala.Predef.String): ControlException = copy(stackTrace = __v) + def withLanguage(__v: edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage): ControlException = copy(language = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -1724,17 +1724,17 @@ final case class ControlError( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError - // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.ControlError]) + def companion: edu.uci.ics.amber.engine.architecture.rpc.ControlException.type = edu.uci.ics.amber.engine.architecture.rpc.ControlException + // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.ControlException]) } -object ControlError extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError = { +object ControlException extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ControlException] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ControlException] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.ControlException = { var __errorMessage: _root_.scala.Predef.String = "" var __errorDetails: _root_.scala.Predef.String = "" var __stackTrace: _root_.scala.Predef.String = "" - var __language: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage.PYTHON + var __language: edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage = edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage.PYTHON var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -1747,48 +1747,48 @@ object ControlError extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber. case 26 => __stackTrace = _input__.readStringRequireUtf8() case 32 => - __language = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage.fromValue(_input__.readEnum()) + __language = edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage.fromValue(_input__.readEnum()) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError( + edu.uci.ics.amber.engine.architecture.rpc.ControlException( errorMessage = __errorMessage, errorDetails = __errorDetails, stackTrace = __stackTrace, language = __language ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.ControlException] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError( + edu.uci.ics.amber.engine.architecture.rpc.ControlException( errorMessage = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), errorDetails = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), stackTrace = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), - language = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage.PYTHON.scalaValueDescriptor).number) + language = edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage.PYTHON.scalaValueDescriptor).number) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(2) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(2) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(2) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(2) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = { (__fieldNumber: @_root_.scala.unchecked) match { - case 4 => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage + case 4 => edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage } } - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.ControlException( errorMessage = "", errorDetails = "", stackTrace = "", - language = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage.PYTHON + language = edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage.PYTHON ) - implicit class ControlErrorLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError](_l) { + implicit class ControlExceptionLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlException]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlException](_l) { def errorMessage: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.errorMessage)((c_, f_) => c_.copy(errorMessage = f_)) def errorDetails: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.errorDetails)((c_, f_) => c_.copy(errorDetails = f_)) def stackTrace: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.stackTrace)((c_, f_) => c_.copy(stackTrace = f_)) - def language: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage] = field(_.language)((c_, f_) => c_.copy(language = f_)) + def language: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage] = field(_.language)((c_, f_) => c_.copy(language = f_)) } final val ERRORMESSAGE_FIELD_NUMBER = 1 final val ERRORDETAILS_FIELD_NUMBER = 2 @@ -1798,32 +1798,32 @@ object ControlError extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber. errorMessage: _root_.scala.Predef.String, errorDetails: _root_.scala.Predef.String, stackTrace: _root_.scala.Predef.String, - language: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError( + language: edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.ControlException = _root_.edu.uci.ics.amber.engine.architecture.rpc.ControlException( errorMessage, errorDetails, stackTrace, language ) - // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ControlError]) + // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ControlException]) } @SerialVersionUID(0L) final case class EmptyReturn( - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty with scalapb.lenses.Updatable[EmptyReturn] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty with scalapb.lenses.Updatable[EmptyReturn] { final override def serializedSize: _root_.scala.Int = 0 def writeTo(`_output__`: _root_.com.google.protobuf.CodedOutputStream): _root_.scala.Unit = { } def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = throw new MatchError(__fieldNumber) def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = throw new MatchError(__field) def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn + def companion: edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn.type = edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) } -object EmptyReturn extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { +object EmptyReturn extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -1832,27 +1832,27 @@ object EmptyReturn extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.e case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn( + edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn( ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn( + edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn( ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(1) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(1) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(1) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(1) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn( ) - implicit class EmptyReturnLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn](_l) { + implicit class EmptyReturnLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn](_l) { } def of( - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = _root_.edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn( ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) } @@ -1860,7 +1860,7 @@ object EmptyReturn extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.e @SerialVersionUID(0L) final case class StringResponse( value: _root_.scala.Predef.String - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty with scalapb.lenses.Updatable[StringResponse] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty with scalapb.lenses.Updatable[StringResponse] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -1907,13 +1907,13 @@ final case class StringResponse( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse + def companion: edu.uci.ics.amber.engine.architecture.rpc.StringResponse.type = edu.uci.ics.amber.engine.architecture.rpc.StringResponse // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]) } -object StringResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse = { +object StringResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.StringResponse = { var __value: _root_.scala.Predef.String = "" var _done__ = false while (!_done__) { @@ -1925,33 +1925,33 @@ object StringResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.ambe case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse( + edu.uci.ics.amber.engine.architecture.rpc.StringResponse( value = __value ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse( + edu.uci.ics.amber.engine.architecture.rpc.StringResponse( value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(4) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(4) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(4) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(4) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.StringResponse( value = "" ) - implicit class StringResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse](_l) { + implicit class StringResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.StringResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.StringResponse](_l) { def value: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.value)((c_, f_) => c_.copy(value = f_)) } final val VALUE_FIELD_NUMBER = 1 def of( value: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.StringResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.StringResponse( value ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]) @@ -1960,7 +1960,7 @@ object StringResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.ambe @SerialVersionUID(0L) final case class IntResponse( value: _root_.scala.Int - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn.NonEmpty with scalapb.lenses.Updatable[IntResponse] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.rpc.ControlReturn.NonEmpty with scalapb.lenses.Updatable[IntResponse] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -2007,13 +2007,13 @@ final case class IntResponse( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse + def companion: edu.uci.ics.amber.engine.architecture.rpc.IntResponse.type = edu.uci.ics.amber.engine.architecture.rpc.IntResponse // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.IntResponse]) } -object IntResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse = { +object IntResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.IntResponse = { var __value: _root_.scala.Int = 0 var _done__ = false while (!_done__) { @@ -2025,33 +2025,33 @@ object IntResponse extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.e case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse( + edu.uci.ics.amber.engine.architecture.rpc.IntResponse( value = __value ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse( + edu.uci.ics.amber.engine.architecture.rpc.IntResponse( value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Int]).getOrElse(0) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(5) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(5) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(5) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(5) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.IntResponse( value = 0 ) - implicit class IntResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse](_l) { + implicit class IntResponseLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.IntResponse]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.IntResponse](_l) { def value: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.value)((c_, f_) => c_.copy(value = f_)) } final val VALUE_FIELD_NUMBER = 1 def of( value: _root_.scala.Int - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.IntResponse = _root_.edu.uci.ics.amber.engine.architecture.rpc.IntResponse( value ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.IntResponse]) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlReturnsProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlReturnsProto.scala new file mode 100644 index 00000000000..9ffc249371a --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControlReturnsProto.scala @@ -0,0 +1,111 @@ +// Generated by the Scala Plugin for the Protocol Buffer Compiler. +// Do not edit! +// +// Protofile syntax: PROTO3 + +package edu.uci.ics.amber.engine.architecture.rpc + +object ControlReturnsProto extends _root_.scalapb.GeneratedFileObject { + lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( + edu.uci.ics.amber.engine.architecture.worker.StatisticsProto, + edu.uci.ics.amber.engine.common.WorkflowMetricsProto, + scalapb.options.ScalapbProto + ) + lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = + Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage, + edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn, + edu.uci.ics.amber.engine.architecture.rpc.ControlException, + edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation, + edu.uci.ics.amber.engine.architecture.rpc.StringResponse, + edu.uci.ics.amber.engine.architecture.rpc.IntResponse, + edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse, + edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse, + edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse, + edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse, + edu.uci.ics.amber.engine.architecture.rpc.TypedValue, + edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue, + edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse, + edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse, + edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse, + edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse + ) + private lazy val ProtoBytes: _root_.scala.Array[Byte] = + scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( + """Cj9lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy9jb250cm9sX3JldHVybnMucHJvdG8SKWVkdS51Y + 2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjGj1lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL + 3dvcmtlci9zdGF0aXN0aWNzLnByb3RvGjZlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3dvcmtmbG93X21ldHJpY3Muc + HJvdG8aFXNjYWxhcGIvc2NhbGFwYi5wcm90byK9DgoNQ29udHJvbFJldHVybhK3AQogcmV0cmlldmVfd29ya2Zsb3dfc3RhdGVfc + mVzcG9uc2UYASABKAsySC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5SZXRyaWV2ZVdvcmtmbG93U + 3RhdGVSZXNwb25zZUIi4j8fEh1yZXRyaWV2ZVdvcmtmbG93U3RhdGVSZXNwb25zZUgAUh1yZXRyaWV2ZVdvcmtmbG93U3RhdGVSZ + XNwb25zZRK7AQohcHJvcGFnYXRlX2NoYW5uZWxfbWFya2VyX3Jlc3BvbnNlGAIgASgLMkkuZWR1LnVjaS5pY3MuYW1iZXIuZW5na + W5lLmFyY2hpdGVjdHVyZS5ycGMuUHJvcGFnYXRlQ2hhbm5lbE1hcmtlclJlc3BvbnNlQiPiPyASHnByb3BhZ2F0ZUNoYW5uZWxNY + XJrZXJSZXNwb25zZUgAUh5wcm9wYWdhdGVDaGFubmVsTWFya2VyUmVzcG9uc2USswEKH3Rha2VfZ2xvYmFsX2NoZWNrcG9pbnRfc + mVzcG9uc2UYAyABKAsyRy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5UYWtlR2xvYmFsQ2hlY2twb + 2ludFJlc3BvbnNlQiHiPx4SHHRha2VHbG9iYWxDaGVja3BvaW50UmVzcG9uc2VIAFIcdGFrZUdsb2JhbENoZWNrcG9pbnRSZXNwb + 25zZRLDAQojZXZhbHVhdGVfcHl0aG9uX2V4cHJlc3Npb25fcmVzcG9uc2UYBCABKAsySy5lZHUudWNpLmljcy5hbWJlci5lbmdpb + mUuYXJjaGl0ZWN0dXJlLnJwYy5FdmFsdWF0ZVB5dGhvbkV4cHJlc3Npb25SZXNwb25zZUIl4j8iEiBldmFsdWF0ZVB5dGhvbkV4c + HJlc3Npb25SZXNwb25zZUgAUiBldmFsdWF0ZVB5dGhvbkV4cHJlc3Npb25SZXNwb25zZRKWAQoXc3RhcnRfd29ya2Zsb3dfcmVzc + G9uc2UYBSABKAsyQC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5TdGFydFdvcmtmbG93UmVzcG9uc + 2VCGuI/FxIVc3RhcnRXb3JrZmxvd1Jlc3BvbnNlSABSFXN0YXJ0V29ya2Zsb3dSZXNwb25zZRKOAQoVd29ya2VyX3N0YXRlX3Jlc + 3BvbnNlGDIgASgLMj4uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuV29ya2VyU3RhdGVSZXNwb25zZ + UIY4j8VEhN3b3JrZXJTdGF0ZVJlc3BvbnNlSABSE3dvcmtlclN0YXRlUmVzcG9uc2USlgEKF3dvcmtlcl9tZXRyaWNzX3Jlc3Bvb + nNlGDMgASgLMkAuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuV29ya2VyTWV0cmljc1Jlc3BvbnNlQ + hriPxcSFXdvcmtlck1ldHJpY3NSZXNwb25zZUgAUhV3b3JrZXJNZXRyaWNzUmVzcG9uc2USqgEKHGZpbmFsaXplX2NoZWNrcG9pb + nRfcmVzcG9uc2UYNCABKAsyRS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5GaW5hbGl6ZUNoZWNrc + G9pbnRSZXNwb25zZUIf4j8cEhpmaW5hbGl6ZUNoZWNrcG9pbnRSZXNwb25zZUgAUhpmaW5hbGl6ZUNoZWNrcG9pbnRSZXNwb25zZ + RJfCgVlcnJvchhlIAEoCzI7LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkNvbnRyb2xFeGNlcHRpb + 25CCuI/BxIFZXJyb3JIAFIFZXJyb3ISbQoMZW1wdHlfcmV0dXJuGGYgASgLMjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY + 2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm5CEOI/DRILZW1wdHlSZXR1cm5IAFILZW1wdHlSZXR1cm4SeQoPc3RyaW5nX3Jlc3Bvb + nNlGGcgASgLMjkuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuU3RyaW5nUmVzcG9uc2VCE+I/EBIOc + 3RyaW5nUmVzcG9uc2VIAFIOc3RyaW5nUmVzcG9uc2USbQoMaW50X3Jlc3BvbnNlGGggASgLMjYuZWR1LnVjaS5pY3MuYW1iZXIuZ + W5naW5lLmFyY2hpdGVjdHVyZS5ycGMuSW50UmVzcG9uc2VCEOI/DRILaW50UmVzcG9uc2VIAFILaW50UmVzcG9uc2VCDgoMc2Vhb + GVkX3ZhbHVlIg0KC0VtcHR5UmV0dXJuIpYCChBDb250cm9sRXhjZXB0aW9uEjUKDGVycm9yTWVzc2FnZRgBIAEoCUIR4j8OEgxlc + nJvck1lc3NhZ2VSDGVycm9yTWVzc2FnZRI1CgxlcnJvckRldGFpbHMYAiABKAlCEeI/DhIMZXJyb3JEZXRhaWxzUgxlcnJvckRld + GFpbHMSLwoKc3RhY2tUcmFjZRgDIAEoCUIP4j8MEgpzdGFja1RyYWNlUgpzdGFja1RyYWNlEmMKCGxhbmd1YWdlGAQgASgOMjguZ + WR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRXJyb3JMYW5ndWFnZUIN4j8KEghsYW5ndWFnZVIIbGFuZ + 3VhZ2Ui8wEKEFJldHVybkludm9jYXRpb24SLAoJY29tbWFuZElkGAEgASgDQg7iPwsSCWNvbW1hbmRJZFIJY29tbWFuZElkEm8KC + 3JldHVyblZhbHVlGAIgASgLMjguZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuQ29udHJvbFJldHVyb + kIT4j8QEgtyZXR1cm5WYWx1ZfABAVILcmV0dXJuVmFsdWU6QOI/PQo7ZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5hb + WJlcm1lc3NhZ2UuQ29udHJvbFBheWxvYWQiMgoOU3RyaW5nUmVzcG9uc2USIAoFdmFsdWUYASABKAlCCuI/BxIFdmFsdWVSBXZhb + HVlIi8KC0ludFJlc3BvbnNlEiAKBXZhbHVlGAEgASgFQgriPwcSBXZhbHVlUgV2YWx1ZSLmAQodUmV0cmlldmVXb3JrZmxvd1N0Y + XRlUmVzcG9uc2USdQoFc3RhdGUYASADKAsyUy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5SZXRya + WV2ZVdvcmtmbG93U3RhdGVSZXNwb25zZS5TdGF0ZUVudHJ5QgriPwcSBXN0YXRlUgVzdGF0ZRpOCgpTdGF0ZUVudHJ5EhoKA2tle + RgBIAEoCUII4j8FEgNrZXlSA2tleRIgCgV2YWx1ZRgCIAEoCUIK4j8HEgV2YWx1ZVIFdmFsdWU6AjgBIjsKGkZpbmFsaXplQ2hlY + 2twb2ludFJlc3BvbnNlEh0KBHNpemUYASABKANCCeI/BhIEc2l6ZVIEc2l6ZSKtAgoeUHJvcGFnYXRlQ2hhbm5lbE1hcmtlclJlc + 3BvbnNlEn4KB3JldHVybnMYASADKAsyVi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Qcm9wYWdhd + GVDaGFubmVsTWFya2VyUmVzcG9uc2UuUmV0dXJuc0VudHJ5QgziPwkSB3JldHVybnNSB3JldHVybnMaigEKDFJldHVybnNFbnRye + RIaCgNrZXkYASABKAlCCOI/BRIDa2V5UgNrZXkSWgoFdmFsdWUYAiABKAsyOC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJja + Gl0ZWN0dXJlLnJwYy5Db250cm9sUmV0dXJuQgriPwcSBXZhbHVlUgV2YWx1ZToCOAEiTAocVGFrZUdsb2JhbENoZWNrcG9pbnRSZ + XNwb25zZRIsCgl0b3RhbFNpemUYASABKANCDuI/CxIJdG90YWxTaXplUgl0b3RhbFNpemUi9QEKClR5cGVkVmFsdWUSLwoKZXhwc + mVzc2lvbhgBIAEoCUIP4j8MEgpleHByZXNzaW9uUgpleHByZXNzaW9uEioKCXZhbHVlX3JlZhgCIAEoCUIN4j8KEgh2YWx1ZVJlZ + lIIdmFsdWVSZWYSKgoJdmFsdWVfc3RyGAMgASgJQg3iPwoSCHZhbHVlU3RyUgh2YWx1ZVN0chItCgp2YWx1ZV90eXBlGAQgASgJQ + g7iPwsSCXZhbHVlVHlwZVIJdmFsdWVUeXBlEi8KCmV4cGFuZGFibGUYBSABKAhCD+I/DBIKZXhwYW5kYWJsZVIKZXhwYW5kYWJsZ + SLRAQoORXZhbHVhdGVkVmFsdWUSVwoFdmFsdWUYASABKAsyNS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlL + nJwYy5UeXBlZFZhbHVlQgriPwcSBXZhbHVlUgV2YWx1ZRJmCgphdHRyaWJ1dGVzGAIgAygLMjUuZWR1LnVjaS5pY3MuYW1iZXIuZ + W5naW5lLmFyY2hpdGVjdHVyZS5ycGMuVHlwZWRWYWx1ZUIP4j8MEgphdHRyaWJ1dGVzUgphdHRyaWJ1dGVzIoIBCiBFdmFsdWF0Z + VB5dGhvbkV4cHJlc3Npb25SZXNwb25zZRJeCgZ2YWx1ZXMYASADKAsyOS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0Z + WN0dXJlLnJwYy5FdmFsdWF0ZWRWYWx1ZUIL4j8IEgZ2YWx1ZXNSBnZhbHVlcyKPAQoVU3RhcnRXb3JrZmxvd1Jlc3BvbnNlEnYKD + ndvcmtmbG93X3N0YXRlGAEgASgOMjguZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5Xb3JrZmxvd0FnZ3JlZ2F0ZWRTd + GF0ZUIV4j8SEg13b3JrZmxvd1N0YXRl8AEBUg13b3JrZmxvd1N0YXRlInUKE1dvcmtlclN0YXRlUmVzcG9uc2USXgoFc3RhdGUYA + SABKA4yOS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLndvcmtlci5Xb3JrZXJTdGF0ZUIN4j8KEgVzdGF0Z + fABAVIFc3RhdGUifwoVV29ya2VyTWV0cmljc1Jlc3BvbnNlEmYKB21ldHJpY3MYASABKAsyOy5lZHUudWNpLmljcy5hbWJlci5lb + mdpbmUuYXJjaGl0ZWN0dXJlLndvcmtlci5Xb3JrZXJNZXRyaWNzQg/iPwwSB21ldHJpY3PwAQFSB21ldHJpY3MqPwoNRXJyb3JMY + W5ndWFnZRIXCgZQWVRIT04QABoL4j8IEgZQWVRIT04SFQoFU0NBTEEQARoK4j8HEgVTQ0FMQUIL4j8IEAFIAFgAeAFiBnByb3RvM + w==""" + ).mkString) + lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { + val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) + _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) + } + lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { + val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) + com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( + edu.uci.ics.amber.engine.architecture.worker.StatisticsProto.javaDescriptor, + edu.uci.ics.amber.engine.common.WorkflowMetricsProto.javaDescriptor, + scalapb.options.ScalapbProto.javaDescriptor + )) + } + @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") + def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControllerServiceFs2Grpc.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControllerServiceFs2Grpc.scala new file mode 100644 index 00000000000..c3d3515e836 --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControllerServiceFs2Grpc.scala @@ -0,0 +1,124 @@ +package edu.uci.ics.amber.engine.architecture.rpc + +import _root_.cats.syntax.all._ + +trait ControllerServiceFs2Grpc[F[_], A] { + def retrieveWorkflowState(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse] + def propagateChannelMarker(request: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse] + def takeGlobalCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse] + def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse] + def consoleMessageTriggered(request: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def portCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def startWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse] + def resumeWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def pauseWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def workerStateUpdated(request: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def workerExecutionCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def linkWorkers(request: edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def controllerInitiateQueryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def retryWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] +} + +object ControllerServiceFs2Grpc extends _root_.fs2.grpc.GeneratedCompanion[ControllerServiceFs2Grpc] { + + def mkClient[F[_]: _root_.cats.effect.Async, A](dispatcher: _root_.cats.effect.std.Dispatcher[F], channel: _root_.io.grpc.Channel, mkMetadata: A => F[_root_.io.grpc.Metadata], clientOptions: _root_.fs2.grpc.client.ClientOptions): ControllerServiceFs2Grpc[F, A] = new ControllerServiceFs2Grpc[F, A] { + def retrieveWorkflowState(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_RETRIEVE_WORKFLOW_STATE, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def propagateChannelMarker(request: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_PROPAGATE_CHANNEL_MARKER, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def takeGlobalCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_TAKE_GLOBAL_CHECKPOINT, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_DEBUG_COMMAND, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_EVALUATE_PYTHON_EXPRESSION, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def consoleMessageTriggered(request: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_CONSOLE_MESSAGE_TRIGGERED, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def portCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_PORT_COMPLETED, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def startWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_START_WORKFLOW, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def resumeWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_RESUME_WORKFLOW, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def pauseWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_PAUSE_WORKFLOW, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def workerStateUpdated(request: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_WORKER_STATE_UPDATED, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def workerExecutionCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_WORKER_EXECUTION_COMPLETED, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def linkWorkers(request: edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_LINK_WORKERS, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def controllerInitiateQueryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_CONTROLLER_INITIATE_QUERY_STATISTICS, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def retryWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_RETRY_WORKFLOW, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + } + + protected def serviceBinding[F[_]: _root_.cats.effect.Async, A](dispatcher: _root_.cats.effect.std.Dispatcher[F], serviceImpl: ControllerServiceFs2Grpc[F, A], mkCtx: _root_.io.grpc.Metadata => F[A], serverOptions: _root_.fs2.grpc.server.ServerOptions): _root_.io.grpc.ServerServiceDefinition = { + _root_.io.grpc.ServerServiceDefinition + .builder(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.SERVICE) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_RETRIEVE_WORKFLOW_STATE, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.retrieveWorkflowState(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_PROPAGATE_CHANNEL_MARKER, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest, edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.propagateChannelMarker(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_TAKE_GLOBAL_CHECKPOINT, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.takeGlobalCheckpoint(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_DEBUG_COMMAND, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.debugCommand(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_EVALUATE_PYTHON_EXPRESSION, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.evaluatePythonExpression(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_CONSOLE_MESSAGE_TRIGGERED, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.consoleMessageTriggered(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_PORT_COMPLETED, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.portCompleted(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_START_WORKFLOW, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.startWorkflow(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_RESUME_WORKFLOW, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.resumeWorkflow(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_PAUSE_WORKFLOW, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.pauseWorkflow(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_WORKER_STATE_UPDATED, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.workerStateUpdated(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_WORKER_EXECUTION_COMPLETED, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.workerExecutionCompleted(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_LINK_WORKERS, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.linkWorkers(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_CONTROLLER_INITIATE_QUERY_STATISTICS, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.controllerInitiateQueryStatistics(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_RETRY_WORKFLOW, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.retryWorkflow(r, _)))) + .build() + } + +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controllerservice/ControllerServiceGrpc.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControllerServiceGrpc.scala similarity index 56% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controllerservice/ControllerServiceGrpc.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControllerServiceGrpc.scala index c0d23c696ff..4fb221f3561 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controllerservice/ControllerServiceGrpc.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControllerServiceGrpc.scala @@ -3,163 +3,163 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controllerservice +package edu.uci.ics.amber.engine.architecture.rpc object ControllerServiceGrpc { - val METHOD_RETRIEVE_WORKFLOW_STATE: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse] = + val METHOD_RETRIEVE_WORKFLOW_STATE: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "RetrieveWorkflowState")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(0))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(0))) .build() - val METHOD_PROPAGATE_CHANNEL_MARKER: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse] = + val METHOD_PROPAGATE_CHANNEL_MARKER: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest, edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "PropagateChannelMarker")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(1))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(1))) .build() - val METHOD_TAKE_GLOBAL_CHECKPOINT: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse] = + val METHOD_TAKE_GLOBAL_CHECKPOINT: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "TakeGlobalCheckpoint")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(2))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(2))) .build() - val METHOD_DEBUG_COMMAND: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_DEBUG_COMMAND: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "DebugCommand")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(3))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(3))) .build() - val METHOD_EVALUATE_PYTHON_EXPRESSION: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse] = + val METHOD_EVALUATE_PYTHON_EXPRESSION: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "EvaluatePythonExpression")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(4))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(4))) .build() - val METHOD_CONSOLE_MESSAGE_TRIGGERED: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_CONSOLE_MESSAGE_TRIGGERED: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "ConsoleMessageTriggered")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(5))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(5))) .build() - val METHOD_PORT_COMPLETED: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_PORT_COMPLETED: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "PortCompleted")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(6))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(6))) .build() - val METHOD_START_WORKFLOW: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse] = + val METHOD_START_WORKFLOW: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "StartWorkflow")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(7))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(7))) .build() - val METHOD_RESUME_WORKFLOW: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_RESUME_WORKFLOW: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "ResumeWorkflow")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(8))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(8))) .build() - val METHOD_PAUSE_WORKFLOW: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_PAUSE_WORKFLOW: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "PauseWorkflow")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(9))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(9))) .build() - val METHOD_WORKER_STATE_UPDATED: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_WORKER_STATE_UPDATED: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "WorkerStateUpdated")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(10))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(10))) .build() - val METHOD_WORKER_EXECUTION_COMPLETED: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_WORKER_EXECUTION_COMPLETED: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "WorkerExecutionCompleted")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(11))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(11))) .build() - val METHOD_LINK_WORKERS: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_LINK_WORKERS: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "LinkWorkers")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(12))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(12))) .build() - val METHOD_CONTROLLER_INITIATE_QUERY_STATISTICS: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_CONTROLLER_INITIATE_QUERY_STATISTICS: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "ControllerInitiateQueryStatistics")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(13))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(13))) .build() - val METHOD_RETRY_WORKFLOW: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_RETRY_WORKFLOW: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.ControllerService", "RetryWorkflow")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(14))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(14))) .build() val SERVICE: _root_.io.grpc.ServiceDescriptor = _root_.io.grpc.ServiceDescriptor.newBuilder("edu.uci.ics.amber.engine.architecture.rpc.ControllerService") - .setSchemaDescriptor(new _root_.scalapb.grpc.ConcreteProtoFileDescriptorSupplier(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor)) + .setSchemaDescriptor(new _root_.scalapb.grpc.ConcreteProtoFileDescriptorSupplier(edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor)) .addMethod(METHOD_RETRIEVE_WORKFLOW_STATE) .addMethod(METHOD_PROPAGATE_CHANNEL_MARKER) .addMethod(METHOD_TAKE_GLOBAL_CHECKPOINT) @@ -179,131 +179,131 @@ object ControllerServiceGrpc { trait ControllerService extends _root_.scalapb.grpc.AbstractService { override def serviceCompanion = ControllerService - def retrieveWorkflowState(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse] - def propagateChannelMarker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse] - def takeGlobalCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse] - def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse] - def consoleMessageTriggered(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def portCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def startWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse] - def resumeWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def pauseWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def workerStateUpdated(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def workerExecutionCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def linkWorkers(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def controllerInitiateQueryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def retryWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] + def retrieveWorkflowState(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse] + def propagateChannelMarker(request: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse] + def takeGlobalCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse] + def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse] + def consoleMessageTriggered(request: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def portCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def startWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse] + def resumeWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def pauseWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def workerStateUpdated(request: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def workerExecutionCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def linkWorkers(request: edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def controllerInitiateQueryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def retryWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] } object ControllerService extends _root_.scalapb.grpc.ServiceCompanion[ControllerService] { implicit def serviceCompanion: _root_.scalapb.grpc.ServiceCompanion[ControllerService] = this - def javaDescriptor: _root_.com.google.protobuf.Descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.scalaDescriptor.services(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.scalaDescriptor.services(0) def bindService(serviceImpl: ControllerService, executionContext: scala.concurrent.ExecutionContext): _root_.io.grpc.ServerServiceDefinition = _root_.io.grpc.ServerServiceDefinition.builder(SERVICE) .addMethod( METHOD_RETRIEVE_WORKFLOW_STATE, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse]): _root_.scala.Unit = serviceImpl.retrieveWorkflowState(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_PROPAGATE_CHANNEL_MARKER, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest, edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse]): _root_.scala.Unit = serviceImpl.propagateChannelMarker(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_TAKE_GLOBAL_CHECKPOINT, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse]): _root_.scala.Unit = serviceImpl.takeGlobalCheckpoint(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_DEBUG_COMMAND, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.debugCommand(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_EVALUATE_PYTHON_EXPRESSION, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse]): _root_.scala.Unit = serviceImpl.evaluatePythonExpression(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_CONSOLE_MESSAGE_TRIGGERED, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.consoleMessageTriggered(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_PORT_COMPLETED, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.portCompleted(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_START_WORKFLOW, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse]): _root_.scala.Unit = serviceImpl.startWorkflow(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_RESUME_WORKFLOW, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.resumeWorkflow(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_PAUSE_WORKFLOW, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.pauseWorkflow(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_WORKER_STATE_UPDATED, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.workerStateUpdated(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_WORKER_EXECUTION_COMPLETED, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.workerExecutionCompleted(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_LINK_WORKERS, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.linkWorkers(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_CONTROLLER_INITIATE_QUERY_STATISTICS, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.controllerInitiateQueryStatistics(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_RETRY_WORKFLOW, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.retryWorkflow(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) @@ -312,81 +312,81 @@ object ControllerServiceGrpc { trait ControllerServiceBlockingClient { def serviceCompanion = ControllerService - def retrieveWorkflowState(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse - def propagateChannelMarker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse - def takeGlobalCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse - def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse - def consoleMessageTriggered(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def portCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def startWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse - def resumeWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def pauseWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def workerStateUpdated(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def workerExecutionCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def linkWorkers(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def controllerInitiateQueryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def retryWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn + def retrieveWorkflowState(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse + def propagateChannelMarker(request: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest): edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse + def takeGlobalCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest): edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse + def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest): edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse + def consoleMessageTriggered(request: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def portCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def startWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse + def resumeWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def pauseWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def workerStateUpdated(request: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def workerExecutionCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def linkWorkers(request: edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def controllerInitiateQueryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def retryWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn } class ControllerServiceBlockingStub(channel: _root_.io.grpc.Channel, options: _root_.io.grpc.CallOptions = _root_.io.grpc.CallOptions.DEFAULT) extends _root_.io.grpc.stub.AbstractStub[ControllerServiceBlockingStub](channel, options) with ControllerServiceBlockingClient { - override def retrieveWorkflowState(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse = { + override def retrieveWorkflowState(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_RETRIEVE_WORKFLOW_STATE, options, request) } - override def propagateChannelMarker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse = { + override def propagateChannelMarker(request: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest): edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_PROPAGATE_CHANNEL_MARKER, options, request) } - override def takeGlobalCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse = { + override def takeGlobalCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest): edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_TAKE_GLOBAL_CHECKPOINT, options, request) } - override def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_DEBUG_COMMAND, options, request) } - override def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse = { + override def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest): edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_EVALUATE_PYTHON_EXPRESSION, options, request) } - override def consoleMessageTriggered(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def consoleMessageTriggered(request: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_CONSOLE_MESSAGE_TRIGGERED, options, request) } - override def portCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def portCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_PORT_COMPLETED, options, request) } - override def startWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse = { + override def startWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_START_WORKFLOW, options, request) } - override def resumeWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def resumeWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_RESUME_WORKFLOW, options, request) } - override def pauseWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def pauseWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_PAUSE_WORKFLOW, options, request) } - override def workerStateUpdated(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def workerStateUpdated(request: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_WORKER_STATE_UPDATED, options, request) } - override def workerExecutionCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def workerExecutionCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_WORKER_EXECUTION_COMPLETED, options, request) } - override def linkWorkers(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def linkWorkers(request: edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_LINK_WORKERS, options, request) } - override def controllerInitiateQueryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def controllerInitiateQueryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_CONTROLLER_INITIATE_QUERY_STATISTICS, options, request) } - override def retryWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def retryWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_RETRY_WORKFLOW, options, request) } @@ -394,63 +394,63 @@ object ControllerServiceGrpc { } class ControllerServiceStub(channel: _root_.io.grpc.Channel, options: _root_.io.grpc.CallOptions = _root_.io.grpc.CallOptions.DEFAULT) extends _root_.io.grpc.stub.AbstractStub[ControllerServiceStub](channel, options) with ControllerService { - override def retrieveWorkflowState(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse] = { + override def retrieveWorkflowState(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.RetrieveWorkflowStateResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_RETRIEVE_WORKFLOW_STATE, options, request) } - override def propagateChannelMarker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse] = { + override def propagateChannelMarker(request: edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.PropagateChannelMarkerResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_PROPAGATE_CHANNEL_MARKER, options, request) } - override def takeGlobalCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse] = { + override def takeGlobalCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.TakeGlobalCheckpointResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_TAKE_GLOBAL_CHECKPOINT, options, request) } - override def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_DEBUG_COMMAND, options, request) } - override def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse] = { + override def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_EVALUATE_PYTHON_EXPRESSION, options, request) } - override def consoleMessageTriggered(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def consoleMessageTriggered(request: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessageTriggeredRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_CONSOLE_MESSAGE_TRIGGERED, options, request) } - override def portCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def portCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.PortCompletedRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_PORT_COMPLETED, options, request) } - override def startWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse] = { + override def startWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StartWorkflowResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_START_WORKFLOW, options, request) } - override def resumeWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def resumeWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_RESUME_WORKFLOW, options, request) } - override def pauseWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def pauseWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_PAUSE_WORKFLOW, options, request) } - override def workerStateUpdated(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def workerStateUpdated(request: edu.uci.ics.amber.engine.architecture.rpc.WorkerStateUpdatedRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_WORKER_STATE_UPDATED, options, request) } - override def workerExecutionCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def workerExecutionCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_WORKER_EXECUTION_COMPLETED, options, request) } - override def linkWorkers(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def linkWorkers(request: edu.uci.ics.amber.engine.architecture.rpc.LinkWorkersRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_LINK_WORKERS, options, request) } - override def controllerInitiateQueryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def controllerInitiateQueryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.QueryStatisticsRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_CONTROLLER_INITIATE_QUERY_STATISTICS, options, request) } - override def retryWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def retryWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.RetryWorkflowRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_RETRY_WORKFLOW, options, request) } @@ -469,6 +469,6 @@ object ControllerServiceGrpc { def stub(channel: _root_.io.grpc.Channel): ControllerServiceStub = new ControllerServiceStub(channel) - def javaDescriptor: _root_.com.google.protobuf.Descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerserviceProto.javaDescriptor.getServices().get(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceProto.javaDescriptor.getServices().get(0) } \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControllerServiceProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControllerServiceProto.scala new file mode 100644 index 00000000000..f255d002706 --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ControllerServiceProto.scala @@ -0,0 +1,67 @@ +// Generated by the Scala Plugin for the Protocol Buffer Compiler. +// Do not edit! +// +// Protofile syntax: PROTO3 + +package edu.uci.ics.amber.engine.architecture.rpc + +object ControllerServiceProto extends _root_.scalapb.GeneratedFileObject { + lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( + edu.uci.ics.amber.engine.architecture.rpc.ControlCommandsProto, + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnsProto, + scalapb.options.ScalapbProto + ) + lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty + private lazy val ProtoBytes: _root_.scala.Array[Byte] = + scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( + """CkJlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy9jb250cm9sbGVyX3NlcnZpY2UucHJvdG8SKWVkd + S51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjGkBlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0d + XJlL3JwYy9jb250cm9sX2NvbW1hbmRzLnByb3RvGj9lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy9jb + 250cm9sX3JldHVybnMucHJvdG8aFXNjYWxhcGIvc2NhbGFwYi5wcm90bzLdEQoRQ29udHJvbGxlclNlcnZpY2USmgEKFVJldHJpZ + XZlV29ya2Zsb3dTdGF0ZRI3LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmVxdWVzdBpIL + mVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlJldHJpZXZlV29ya2Zsb3dTdGF0ZVJlc3BvbnNlEq0BC + hZQcm9wYWdhdGVDaGFubmVsTWFya2VyEkguZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuUHJvcGFnY + XRlQ2hhbm5lbE1hcmtlclJlcXVlc3QaSS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Qcm9wYWdhd + GVDaGFubmVsTWFya2VyUmVzcG9uc2USpwEKFFRha2VHbG9iYWxDaGVja3BvaW50EkYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lL + mFyY2hpdGVjdHVyZS5ycGMuVGFrZUdsb2JhbENoZWNrcG9pbnRSZXF1ZXN0GkcuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY + 2hpdGVjdHVyZS5ycGMuVGFrZUdsb2JhbENoZWNrcG9pbnRSZXNwb25zZRKGAQoMRGVidWdDb21tYW5kEj4uZWR1LnVjaS5pY3MuY + W1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRGVidWdDb21tYW5kUmVxdWVzdBo2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZ + S5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmV0dXJuErMBChhFdmFsdWF0ZVB5dGhvbkV4cHJlc3Npb24SSi5lZHUudWNpLmljcy5hb + WJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FdmFsdWF0ZVB5dGhvbkV4cHJlc3Npb25SZXF1ZXN0GksuZWR1LnVjaS5pY3MuY + W1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRXZhbHVhdGVQeXRob25FeHByZXNzaW9uUmVzcG9uc2USnAEKF0NvbnNvbGVNZ + XNzYWdlVHJpZ2dlcmVkEkkuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuQ29uc29sZU1lc3NhZ2VUc + mlnZ2VyZWRSZXF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm4SiAEKD + VBvcnRDb21wbGV0ZWQSPy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Qb3J0Q29tcGxldGVkUmVxd + WVzdBo2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmV0dXJuEooBCg1TdGFydFdvcmtmb + G93EjcuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXF1ZXN0GkAuZWR1LnVjaS5pY3MuY + W1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuU3RhcnRXb3JrZmxvd1Jlc3BvbnNlEoEBCg5SZXN1bWVXb3JrZmxvdxI3LmVkd + S51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmVxdWVzdBo2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ + 2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmV0dXJuEoABCg1QYXVzZVdvcmtmbG93EjcuZWR1LnVjaS5pY3MuYW1iZXIuZW5na + W5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5yc + GMuRW1wdHlSZXR1cm4SkgEKEldvcmtlclN0YXRlVXBkYXRlZBJELmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1c + mUucnBjLldvcmtlclN0YXRlVXBkYXRlZFJlcXVlc3QaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwY + y5FbXB0eVJldHVybhKLAQoYV29ya2VyRXhlY3V0aW9uQ29tcGxldGVkEjcuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpd + GVjdHVyZS5ycGMuRW1wdHlSZXF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZ + XR1cm4ShAEKC0xpbmtXb3JrZXJzEj0uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuTGlua1dvcmtlc + nNSZXF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm4SngEKIUNvbnRyb + 2xsZXJJbml0aWF0ZVF1ZXJ5U3RhdGlzdGljcxJBLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlF1Z + XJ5U3RhdGlzdGljc1JlcXVlc3QaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJldHVyb + hKIAQoNUmV0cnlXb3JrZmxvdxI/LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlJldHJ5V29ya2Zsb + 3dSZXF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm5CC+I/CBABSABYA + HgBYgZwcm90bzM=""" + ).mkString) + lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { + val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) + _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) + } + lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { + val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) + com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( + edu.uci.ics.amber.engine.architecture.rpc.ControlCommandsProto.javaDescriptor, + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnsProto.javaDescriptor, + scalapb.options.ScalapbProto.javaDescriptor + )) + } + @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") + def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ErrorLanguage.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ErrorLanguage.scala similarity index 77% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ErrorLanguage.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ErrorLanguage.scala index 6e28d9688bd..115f3fe027c 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ErrorLanguage.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ErrorLanguage.scala @@ -3,14 +3,14 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlreturns +package edu.uci.ics.amber.engine.architecture.rpc sealed abstract class ErrorLanguage(val value: _root_.scala.Int) extends _root_.scalapb.GeneratedEnum { type EnumType = ErrorLanguage def isPython: _root_.scala.Boolean = false def isScala: _root_.scala.Boolean = false - def companion: _root_.scalapb.GeneratedEnumCompanion[ErrorLanguage] = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage - final def asRecognized: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage.Recognized] = if (isUnrecognized) _root_.scala.None else _root_.scala.Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ErrorLanguage.Recognized]) + def companion: _root_.scalapb.GeneratedEnumCompanion[ErrorLanguage] = edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage + final def asRecognized: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage.Recognized] = if (isUnrecognized) _root_.scala.None else _root_.scala.Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.ErrorLanguage.Recognized]) } object ErrorLanguage extends _root_.scalapb.GeneratedEnumCompanion[ErrorLanguage] { @@ -39,6 +39,6 @@ object ErrorLanguage extends _root_.scalapb.GeneratedEnumCompanion[ErrorLanguage case 1 => SCALA case __other => Unrecognized(__other) } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.EnumDescriptor = ControlreturnsProto.javaDescriptor.getEnumTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.EnumDescriptor = ControlreturnsProto.scalaDescriptor.enums(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.EnumDescriptor = ControlReturnsProto.javaDescriptor.getEnumTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.EnumDescriptor = ControlReturnsProto.scalaDescriptor.enums(0) } \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/EvaluatedValue.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/EvaluatedValue.scala similarity index 66% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/EvaluatedValue.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/EvaluatedValue.scala index c82fefe7790..e7924656be2 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/EvaluatedValue.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/EvaluatedValue.scala @@ -3,12 +3,12 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlreturns +package edu.uci.ics.amber.engine.architecture.rpc @SerialVersionUID(0L) final case class EvaluatedValue( - value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue], - attributes: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue] + value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.TypedValue], + attributes: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.TypedValue] ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[EvaluatedValue] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -47,13 +47,13 @@ final case class EvaluatedValue( __m.writeTo(_output__) }; } - def getValue: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue = value.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue.defaultInstance) + def getValue: edu.uci.ics.amber.engine.architecture.rpc.TypedValue = value.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.TypedValue.defaultInstance) def clearValue: EvaluatedValue = copy(value = _root_.scala.None) - def withValue(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue): EvaluatedValue = copy(value = Option(__v)) + def withValue(__v: edu.uci.ics.amber.engine.architecture.rpc.TypedValue): EvaluatedValue = copy(value = Option(__v)) def clearAttributes = copy(attributes = _root_.scala.Seq.empty) - def addAttributes(__vs: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue *): EvaluatedValue = addAllAttributes(__vs) - def addAllAttributes(__vs: Iterable[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue]): EvaluatedValue = copy(attributes = attributes ++ __vs) - def withAttributes(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue]): EvaluatedValue = copy(attributes = __v) + def addAttributes(__vs: edu.uci.ics.amber.engine.architecture.rpc.TypedValue *): EvaluatedValue = addAllAttributes(__vs) + def addAllAttributes(__vs: Iterable[edu.uci.ics.amber.engine.architecture.rpc.TypedValue]): EvaluatedValue = copy(attributes = attributes ++ __vs) + def withAttributes(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.TypedValue]): EvaluatedValue = copy(attributes = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => value.orNull @@ -68,68 +68,68 @@ final case class EvaluatedValue( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue + def companion: edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue.type = edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]) } -object EvaluatedValue extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue = { - var __value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue] = _root_.scala.None - val __attributes: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue] +object EvaluatedValue extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue = { + var __value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.TypedValue] = _root_.scala.None + val __attributes: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.TypedValue] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.TypedValue] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __value = Option(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __value = Option(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.TypedValue](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => - __attributes += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue](_input__) + __attributes += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.TypedValue](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue( + edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue( value = __value, attributes = __attributes.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue( - value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue]]), - attributes = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue]]).getOrElse(_root_.scala.Seq.empty) + edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue( + value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.TypedValue]]), + attributes = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.TypedValue]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(11) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(11) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(11) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(11) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue - case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue + case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.TypedValue + case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.TypedValue } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue( value = _root_.scala.None, attributes = _root_.scala.Seq.empty ) - implicit class EvaluatedValueLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue](_l) { - def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue] = field(_.getValue)((c_, f_) => c_.copy(value = Option(f_))) - def optionalValue: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue]] = field(_.value)((c_, f_) => c_.copy(value = f_)) - def attributes: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue]] = field(_.attributes)((c_, f_) => c_.copy(attributes = f_)) + implicit class EvaluatedValueLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue](_l) { + def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.TypedValue] = field(_.getValue)((c_, f_) => c_.copy(value = Option(f_))) + def optionalValue: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.TypedValue]] = field(_.value)((c_, f_) => c_.copy(value = f_)) + def attributes: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.TypedValue]] = field(_.attributes)((c_, f_) => c_.copy(attributes = f_)) } final val VALUE_FIELD_NUMBER = 1 final val ATTRIBUTES_FIELD_NUMBER = 2 def of( - value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue], - attributes: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue] - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue( + value: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.TypedValue], + attributes: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.TypedValue] + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue = _root_.edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue( value, attributes ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/RPCTesterFs2Grpc.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/RPCTesterFs2Grpc.scala new file mode 100644 index 00000000000..dee1efeb76d --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/RPCTesterFs2Grpc.scala @@ -0,0 +1,89 @@ +package edu.uci.ics.amber.engine.architecture.rpc + +import _root_.cats.syntax.all._ + +trait RPCTesterFs2Grpc[F[_], A] { + def sendPing(request: edu.uci.ics.amber.engine.architecture.rpc.Ping, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] + def sendPong(request: edu.uci.ics.amber.engine.architecture.rpc.Pong, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] + def sendNested(request: edu.uci.ics.amber.engine.architecture.rpc.Nested, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] + def sendPass(request: edu.uci.ics.amber.engine.architecture.rpc.Pass, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] + def sendErrorCommand(request: edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] + def sendRecursion(request: edu.uci.ics.amber.engine.architecture.rpc.Recursion, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] + def sendCollect(request: edu.uci.ics.amber.engine.architecture.rpc.Collect, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] + def sendGenerateNumber(request: edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] + def sendMultiCall(request: edu.uci.ics.amber.engine.architecture.rpc.MultiCall, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] + def sendChain(request: edu.uci.ics.amber.engine.architecture.rpc.Chain, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] +} + +object RPCTesterFs2Grpc extends _root_.fs2.grpc.GeneratedCompanion[RPCTesterFs2Grpc] { + + def mkClient[F[_]: _root_.cats.effect.Async, A](dispatcher: _root_.cats.effect.std.Dispatcher[F], channel: _root_.io.grpc.Channel, mkMetadata: A => F[_root_.io.grpc.Metadata], clientOptions: _root_.fs2.grpc.client.ClientOptions): RPCTesterFs2Grpc[F, A] = new RPCTesterFs2Grpc[F, A] { + def sendPing(request: edu.uci.ics.amber.engine.architecture.rpc.Ping, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_PING, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def sendPong(request: edu.uci.ics.amber.engine.architecture.rpc.Pong, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_PONG, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def sendNested(request: edu.uci.ics.amber.engine.architecture.rpc.Nested, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_NESTED, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def sendPass(request: edu.uci.ics.amber.engine.architecture.rpc.Pass, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_PASS, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def sendErrorCommand(request: edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_ERROR_COMMAND, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def sendRecursion(request: edu.uci.ics.amber.engine.architecture.rpc.Recursion, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_RECURSION, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def sendCollect(request: edu.uci.ics.amber.engine.architecture.rpc.Collect, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_COLLECT, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def sendGenerateNumber(request: edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_GENERATE_NUMBER, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def sendMultiCall(request: edu.uci.ics.amber.engine.architecture.rpc.MultiCall, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_MULTI_CALL, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def sendChain(request: edu.uci.ics.amber.engine.architecture.rpc.Chain, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_CHAIN, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + } + + protected def serviceBinding[F[_]: _root_.cats.effect.Async, A](dispatcher: _root_.cats.effect.std.Dispatcher[F], serviceImpl: RPCTesterFs2Grpc[F, A], mkCtx: _root_.io.grpc.Metadata => F[A], serverOptions: _root_.fs2.grpc.server.ServerOptions): _root_.io.grpc.ServerServiceDefinition = { + _root_.io.grpc.ServerServiceDefinition + .builder(edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.SERVICE) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_PING, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.Ping, edu.uci.ics.amber.engine.architecture.rpc.IntResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendPing(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_PONG, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.Pong, edu.uci.ics.amber.engine.architecture.rpc.IntResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendPong(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_NESTED, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.Nested, edu.uci.ics.amber.engine.architecture.rpc.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendNested(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_PASS, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.Pass, edu.uci.ics.amber.engine.architecture.rpc.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendPass(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_ERROR_COMMAND, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand, edu.uci.ics.amber.engine.architecture.rpc.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendErrorCommand(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_RECURSION, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.Recursion, edu.uci.ics.amber.engine.architecture.rpc.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendRecursion(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_COLLECT, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.Collect, edu.uci.ics.amber.engine.architecture.rpc.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendCollect(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_GENERATE_NUMBER, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber, edu.uci.ics.amber.engine.architecture.rpc.IntResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendGenerateNumber(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_MULTI_CALL, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.MultiCall, edu.uci.ics.amber.engine.architecture.rpc.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendMultiCall(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.METHOD_SEND_CHAIN, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.Chain, edu.uci.ics.amber.engine.architecture.rpc.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendChain(r, _)))) + .build() + } + +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/testerservice/RPCTesterGrpc.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/RPCTesterGrpc.scala similarity index 59% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/testerservice/RPCTesterGrpc.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/RPCTesterGrpc.scala index 39a6f1f4ce7..8aff7196384 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/testerservice/RPCTesterGrpc.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/RPCTesterGrpc.scala @@ -3,113 +3,113 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.testerservice +package edu.uci.ics.amber.engine.architecture.rpc object RPCTesterGrpc { - val METHOD_SEND_PING: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = + val METHOD_SEND_PING: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.Ping, edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.RPCTester", "SendPing")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor.getServices().get(0).getMethods().get(0))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.Ping]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.IntResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor.getServices().get(0).getMethods().get(0))) .build() - val METHOD_SEND_PONG: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = + val METHOD_SEND_PONG: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.Pong, edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.RPCTester", "SendPong")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor.getServices().get(0).getMethods().get(1))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.Pong]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.IntResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor.getServices().get(0).getMethods().get(1))) .build() - val METHOD_SEND_NESTED: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = + val METHOD_SEND_NESTED: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.Nested, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.RPCTester", "SendNested")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor.getServices().get(0).getMethods().get(2))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.Nested]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor.getServices().get(0).getMethods().get(2))) .build() - val METHOD_SEND_PASS: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = + val METHOD_SEND_PASS: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.Pass, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.RPCTester", "SendPass")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor.getServices().get(0).getMethods().get(3))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.Pass]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor.getServices().get(0).getMethods().get(3))) .build() - val METHOD_SEND_ERROR_COMMAND: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = + val METHOD_SEND_ERROR_COMMAND: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.RPCTester", "SendErrorCommand")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor.getServices().get(0).getMethods().get(4))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor.getServices().get(0).getMethods().get(4))) .build() - val METHOD_SEND_RECURSION: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = + val METHOD_SEND_RECURSION: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.Recursion, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.RPCTester", "SendRecursion")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor.getServices().get(0).getMethods().get(5))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.Recursion]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor.getServices().get(0).getMethods().get(5))) .build() - val METHOD_SEND_COLLECT: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = + val METHOD_SEND_COLLECT: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.Collect, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.RPCTester", "SendCollect")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor.getServices().get(0).getMethods().get(6))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.Collect]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor.getServices().get(0).getMethods().get(6))) .build() - val METHOD_SEND_GENERATE_NUMBER: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = + val METHOD_SEND_GENERATE_NUMBER: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber, edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.RPCTester", "SendGenerateNumber")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor.getServices().get(0).getMethods().get(7))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.IntResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor.getServices().get(0).getMethods().get(7))) .build() - val METHOD_SEND_MULTI_CALL: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = + val METHOD_SEND_MULTI_CALL: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.MultiCall, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.RPCTester", "SendMultiCall")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor.getServices().get(0).getMethods().get(8))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.MultiCall]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor.getServices().get(0).getMethods().get(8))) .build() - val METHOD_SEND_CHAIN: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = + val METHOD_SEND_CHAIN: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.Chain, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.RPCTester", "SendChain")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor.getServices().get(0).getMethods().get(9))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.Chain]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor.getServices().get(0).getMethods().get(9))) .build() val SERVICE: _root_.io.grpc.ServiceDescriptor = _root_.io.grpc.ServiceDescriptor.newBuilder("edu.uci.ics.amber.engine.architecture.rpc.RPCTester") - .setSchemaDescriptor(new _root_.scalapb.grpc.ConcreteProtoFileDescriptorSupplier(edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor)) + .setSchemaDescriptor(new _root_.scalapb.grpc.ConcreteProtoFileDescriptorSupplier(edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor)) .addMethod(METHOD_SEND_PING) .addMethod(METHOD_SEND_PONG) .addMethod(METHOD_SEND_NESTED) @@ -124,91 +124,91 @@ object RPCTesterGrpc { trait RPCTester extends _root_.scalapb.grpc.AbstractService { override def serviceCompanion = RPCTester - def sendPing(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] - def sendPong(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] - def sendNested(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] - def sendPass(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] - def sendErrorCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] - def sendRecursion(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] - def sendCollect(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] - def sendGenerateNumber(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] - def sendMultiCall(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] - def sendChain(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] + def sendPing(request: edu.uci.ics.amber.engine.architecture.rpc.Ping): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] + def sendPong(request: edu.uci.ics.amber.engine.architecture.rpc.Pong): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] + def sendNested(request: edu.uci.ics.amber.engine.architecture.rpc.Nested): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] + def sendPass(request: edu.uci.ics.amber.engine.architecture.rpc.Pass): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] + def sendErrorCommand(request: edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] + def sendRecursion(request: edu.uci.ics.amber.engine.architecture.rpc.Recursion): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] + def sendCollect(request: edu.uci.ics.amber.engine.architecture.rpc.Collect): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] + def sendGenerateNumber(request: edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] + def sendMultiCall(request: edu.uci.ics.amber.engine.architecture.rpc.MultiCall): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] + def sendChain(request: edu.uci.ics.amber.engine.architecture.rpc.Chain): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] } object RPCTester extends _root_.scalapb.grpc.ServiceCompanion[RPCTester] { implicit def serviceCompanion: _root_.scalapb.grpc.ServiceCompanion[RPCTester] = this - def javaDescriptor: _root_.com.google.protobuf.Descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor.getServices().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.scalaDescriptor.services(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor.getServices().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.scalaDescriptor.services(0) def bindService(serviceImpl: RPCTester, executionContext: scala.concurrent.ExecutionContext): _root_.io.grpc.ServerServiceDefinition = _root_.io.grpc.ServerServiceDefinition.builder(SERVICE) .addMethod( METHOD_SEND_PING, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.Ping, edu.uci.ics.amber.engine.architecture.rpc.IntResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.Ping, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.IntResponse]): _root_.scala.Unit = serviceImpl.sendPing(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_SEND_PONG, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.Pong, edu.uci.ics.amber.engine.architecture.rpc.IntResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.Pong, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.IntResponse]): _root_.scala.Unit = serviceImpl.sendPong(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_SEND_NESTED, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.Nested, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.Nested, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]): _root_.scala.Unit = serviceImpl.sendNested(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_SEND_PASS, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.Pass, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.Pass, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]): _root_.scala.Unit = serviceImpl.sendPass(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_SEND_ERROR_COMMAND, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]): _root_.scala.Unit = serviceImpl.sendErrorCommand(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_SEND_RECURSION, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.Recursion, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.Recursion, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]): _root_.scala.Unit = serviceImpl.sendRecursion(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_SEND_COLLECT, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.Collect, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.Collect, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]): _root_.scala.Unit = serviceImpl.sendCollect(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_SEND_GENERATE_NUMBER, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber, edu.uci.ics.amber.engine.architecture.rpc.IntResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.IntResponse]): _root_.scala.Unit = serviceImpl.sendGenerateNumber(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_SEND_MULTI_CALL, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.MultiCall, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.MultiCall, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]): _root_.scala.Unit = serviceImpl.sendMultiCall(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_SEND_CHAIN, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.Chain, edu.uci.ics.amber.engine.architecture.rpc.StringResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.Chain, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.StringResponse]): _root_.scala.Unit = serviceImpl.sendChain(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) @@ -217,56 +217,56 @@ object RPCTesterGrpc { trait RPCTesterBlockingClient { def serviceCompanion = RPCTester - def sendPing(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse - def sendPong(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse - def sendNested(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse - def sendPass(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse - def sendErrorCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse - def sendRecursion(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse - def sendCollect(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse - def sendGenerateNumber(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse - def sendMultiCall(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse - def sendChain(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse + def sendPing(request: edu.uci.ics.amber.engine.architecture.rpc.Ping): edu.uci.ics.amber.engine.architecture.rpc.IntResponse + def sendPong(request: edu.uci.ics.amber.engine.architecture.rpc.Pong): edu.uci.ics.amber.engine.architecture.rpc.IntResponse + def sendNested(request: edu.uci.ics.amber.engine.architecture.rpc.Nested): edu.uci.ics.amber.engine.architecture.rpc.StringResponse + def sendPass(request: edu.uci.ics.amber.engine.architecture.rpc.Pass): edu.uci.ics.amber.engine.architecture.rpc.StringResponse + def sendErrorCommand(request: edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand): edu.uci.ics.amber.engine.architecture.rpc.StringResponse + def sendRecursion(request: edu.uci.ics.amber.engine.architecture.rpc.Recursion): edu.uci.ics.amber.engine.architecture.rpc.StringResponse + def sendCollect(request: edu.uci.ics.amber.engine.architecture.rpc.Collect): edu.uci.ics.amber.engine.architecture.rpc.StringResponse + def sendGenerateNumber(request: edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber): edu.uci.ics.amber.engine.architecture.rpc.IntResponse + def sendMultiCall(request: edu.uci.ics.amber.engine.architecture.rpc.MultiCall): edu.uci.ics.amber.engine.architecture.rpc.StringResponse + def sendChain(request: edu.uci.ics.amber.engine.architecture.rpc.Chain): edu.uci.ics.amber.engine.architecture.rpc.StringResponse } class RPCTesterBlockingStub(channel: _root_.io.grpc.Channel, options: _root_.io.grpc.CallOptions = _root_.io.grpc.CallOptions.DEFAULT) extends _root_.io.grpc.stub.AbstractStub[RPCTesterBlockingStub](channel, options) with RPCTesterBlockingClient { - override def sendPing(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse = { + override def sendPing(request: edu.uci.ics.amber.engine.architecture.rpc.Ping): edu.uci.ics.amber.engine.architecture.rpc.IntResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_SEND_PING, options, request) } - override def sendPong(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse = { + override def sendPong(request: edu.uci.ics.amber.engine.architecture.rpc.Pong): edu.uci.ics.amber.engine.architecture.rpc.IntResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_SEND_PONG, options, request) } - override def sendNested(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse = { + override def sendNested(request: edu.uci.ics.amber.engine.architecture.rpc.Nested): edu.uci.ics.amber.engine.architecture.rpc.StringResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_SEND_NESTED, options, request) } - override def sendPass(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse = { + override def sendPass(request: edu.uci.ics.amber.engine.architecture.rpc.Pass): edu.uci.ics.amber.engine.architecture.rpc.StringResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_SEND_PASS, options, request) } - override def sendErrorCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse = { + override def sendErrorCommand(request: edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand): edu.uci.ics.amber.engine.architecture.rpc.StringResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_SEND_ERROR_COMMAND, options, request) } - override def sendRecursion(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse = { + override def sendRecursion(request: edu.uci.ics.amber.engine.architecture.rpc.Recursion): edu.uci.ics.amber.engine.architecture.rpc.StringResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_SEND_RECURSION, options, request) } - override def sendCollect(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse = { + override def sendCollect(request: edu.uci.ics.amber.engine.architecture.rpc.Collect): edu.uci.ics.amber.engine.architecture.rpc.StringResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_SEND_COLLECT, options, request) } - override def sendGenerateNumber(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse = { + override def sendGenerateNumber(request: edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber): edu.uci.ics.amber.engine.architecture.rpc.IntResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_SEND_GENERATE_NUMBER, options, request) } - override def sendMultiCall(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse = { + override def sendMultiCall(request: edu.uci.ics.amber.engine.architecture.rpc.MultiCall): edu.uci.ics.amber.engine.architecture.rpc.StringResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_SEND_MULTI_CALL, options, request) } - override def sendChain(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse = { + override def sendChain(request: edu.uci.ics.amber.engine.architecture.rpc.Chain): edu.uci.ics.amber.engine.architecture.rpc.StringResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_SEND_CHAIN, options, request) } @@ -274,43 +274,43 @@ object RPCTesterGrpc { } class RPCTesterStub(channel: _root_.io.grpc.Channel, options: _root_.io.grpc.CallOptions = _root_.io.grpc.CallOptions.DEFAULT) extends _root_.io.grpc.stub.AbstractStub[RPCTesterStub](channel, options) with RPCTester { - override def sendPing(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = { + override def sendPing(request: edu.uci.ics.amber.engine.architecture.rpc.Ping): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_SEND_PING, options, request) } - override def sendPong(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = { + override def sendPong(request: edu.uci.ics.amber.engine.architecture.rpc.Pong): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_SEND_PONG, options, request) } - override def sendNested(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { + override def sendNested(request: edu.uci.ics.amber.engine.architecture.rpc.Nested): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_SEND_NESTED, options, request) } - override def sendPass(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { + override def sendPass(request: edu.uci.ics.amber.engine.architecture.rpc.Pass): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_SEND_PASS, options, request) } - override def sendErrorCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { + override def sendErrorCommand(request: edu.uci.ics.amber.engine.architecture.rpc.ErrorCommand): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_SEND_ERROR_COMMAND, options, request) } - override def sendRecursion(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { + override def sendRecursion(request: edu.uci.ics.amber.engine.architecture.rpc.Recursion): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_SEND_RECURSION, options, request) } - override def sendCollect(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { + override def sendCollect(request: edu.uci.ics.amber.engine.architecture.rpc.Collect): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_SEND_COLLECT, options, request) } - override def sendGenerateNumber(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = { + override def sendGenerateNumber(request: edu.uci.ics.amber.engine.architecture.rpc.GenerateNumber): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.IntResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_SEND_GENERATE_NUMBER, options, request) } - override def sendMultiCall(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { + override def sendMultiCall(request: edu.uci.ics.amber.engine.architecture.rpc.MultiCall): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_SEND_MULTI_CALL, options, request) } - override def sendChain(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { + override def sendChain(request: edu.uci.ics.amber.engine.architecture.rpc.Chain): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.StringResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_SEND_CHAIN, options, request) } @@ -329,6 +329,6 @@ object RPCTesterGrpc { def stub(channel: _root_.io.grpc.Channel): RPCTesterStub = new RPCTesterStub(channel) - def javaDescriptor: _root_.com.google.protobuf.Descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.testerservice.TesterserviceProto.javaDescriptor.getServices().get(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.TesterServiceProto.javaDescriptor.getServices().get(0) } \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ReturnInvocation.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ReturnInvocation.scala similarity index 60% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ReturnInvocation.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ReturnInvocation.scala index 3c016c0fe0d..ab8af101846 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ReturnInvocation.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/ReturnInvocation.scala @@ -3,12 +3,12 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlreturns +package edu.uci.ics.amber.engine.architecture.rpc @SerialVersionUID(0L) final case class ReturnInvocation( commandId: _root_.scala.Long, - returnValue: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn + returnValue: edu.uci.ics.amber.engine.architecture.rpc.ControlReturn ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ReturnInvocation] with edu.uci.ics.amber.engine.common.ambermessage.ControlPayload { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -23,7 +23,7 @@ final case class ReturnInvocation( }; { - val __value = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation._typemapper_returnValue.toBase(returnValue) + val __value = edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation._typemapper_returnValue.toBase(returnValue) if (__value.serializedSize != 0) { __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } @@ -47,7 +47,7 @@ final case class ReturnInvocation( } }; { - val __v = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation._typemapper_returnValue.toBase(returnValue) + val __v = edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation._typemapper_returnValue.toBase(returnValue) if (__v.serializedSize != 0) { _output__.writeTag(2, 2) _output__.writeUInt32NoTag(__v.serializedSize) @@ -56,7 +56,7 @@ final case class ReturnInvocation( }; } def withCommandId(__v: _root_.scala.Long): ReturnInvocation = copy(commandId = __v) - def withReturnValue(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn): ReturnInvocation = copy(returnValue = __v) + def withReturnValue(__v: edu.uci.ics.amber.engine.architecture.rpc.ControlReturn): ReturnInvocation = copy(returnValue = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -64,8 +64,8 @@ final case class ReturnInvocation( if (__t != 0L) __t else null } case 2 => { - val __t = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation._typemapper_returnValue.toBase(returnValue) - if (__t != edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.defaultInstance) __t else null + val __t = edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation._typemapper_returnValue.toBase(returnValue) + if (__t != edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.defaultInstance) __t else null } } } @@ -73,19 +73,19 @@ final case class ReturnInvocation( _root_.scala.Predef.require(__field.containingMessage eq companion.scalaDescriptor) (__field.number: @_root_.scala.unchecked) match { case 1 => _root_.scalapb.descriptors.PLong(commandId) - case 2 => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation._typemapper_returnValue.toBase(returnValue).toPMessage + case 2 => edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation._typemapper_returnValue.toBase(returnValue).toPMessage } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation + def companion: edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation.type = edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation]) } -object ReturnInvocation extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation = { +object ReturnInvocation extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation = { var __commandId: _root_.scala.Long = 0L - var __returnValue: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage] = _root_.scala.None + var __returnValue: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -94,51 +94,51 @@ object ReturnInvocation extends scalapb.GeneratedMessageCompanion[edu.uci.ics.am case 8 => __commandId = _input__.readInt64() case 18 => - __returnValue = _root_.scala.Some(__returnValue.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __returnValue = _root_.scala.Some(__returnValue.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation( + edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation( commandId = __commandId, - returnValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation._typemapper_returnValue.toCustom(__returnValue.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.defaultInstance)) + returnValue = edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation._typemapper_returnValue.toCustom(__returnValue.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.defaultInstance)) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation( + edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation( commandId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Long]).getOrElse(0L), - returnValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation._typemapper_returnValue.toCustom(__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.defaultInstance)) + returnValue = edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation._typemapper_returnValue.toCustom(__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.defaultInstance)) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(3) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(3) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(3) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(3) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage + case 2 => __out = edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation( commandId = 0L, - returnValue = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation._typemapper_returnValue.toCustom(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage.defaultInstance) + returnValue = edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation._typemapper_returnValue.toCustom(edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage.defaultInstance) ) - implicit class ReturnInvocationLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation](_l) { + implicit class ReturnInvocationLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation](_l) { def commandId: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.commandId)((c_, f_) => c_.copy(commandId = f_)) - def returnValue: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn] = field(_.returnValue)((c_, f_) => c_.copy(returnValue = f_)) + def returnValue: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn] = field(_.returnValue)((c_, f_) => c_.copy(returnValue = f_)) } final val COMMANDID_FIELD_NUMBER = 1 final val RETURNVALUE_FIELD_NUMBER = 2 @transient - private[controlreturns] val _typemapper_returnValue: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn]] + private[rpc] val _typemapper_returnValue: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.rpc.ControlReturnMessage, edu.uci.ics.amber.engine.architecture.rpc.ControlReturn]] def of( commandId: _root_.scala.Long, - returnValue: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturn - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation( + returnValue: edu.uci.ics.amber.engine.architecture.rpc.ControlReturn + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation = _root_.edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation( commandId, returnValue ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/TesterServiceProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/TesterServiceProto.scala new file mode 100644 index 00000000000..3f8bf023d86 --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/TesterServiceProto.scala @@ -0,0 +1,54 @@ +// Generated by the Scala Plugin for the Protocol Buffer Compiler. +// Do not edit! +// +// Protofile syntax: PROTO3 + +package edu.uci.ics.amber.engine.architecture.rpc + +object TesterServiceProto extends _root_.scalapb.GeneratedFileObject { + lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( + edu.uci.ics.amber.engine.architecture.rpc.ControlCommandsProto, + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnsProto, + scalapb.options.ScalapbProto + ) + lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty + private lazy val ProtoBytes: _root_.scala.Array[Byte] = + scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( + """Cj5lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy90ZXN0ZXJfc2VydmljZS5wcm90bxIpZWR1LnVja + S5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMaQGVkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9hcmNoaXRlY3R1cmUvc + nBjL2NvbnRyb2xfY29tbWFuZHMucHJvdG8aP2VkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9hcmNoaXRlY3R1cmUvcnBjL2NvbnRyb + 2xfcmV0dXJucy5wcm90bxoVc2NhbGFwYi9zY2FsYXBiLnByb3RvMo4KCglSUENUZXN0ZXISdQoIU2VuZFBpbmcSLy5lZHUudWNpL + mljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5QaW5nGjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjd + HVyZS5ycGMuSW50UmVzcG9uc2UiABJ1CghTZW5kUG9uZxIvLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUuc + nBjLlBvbmcaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5JbnRSZXNwb25zZSIAEnwKClNlbmROZ + XN0ZWQSMS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5OZXN0ZWQaOS5lZHUudWNpLmljcy5hbWJlc + i5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5TdHJpbmdSZXNwb25zZSIAEngKCFNlbmRQYXNzEi8uZWR1LnVjaS5pY3MuYW1iZXIuZ + W5naW5lLmFyY2hpdGVjdHVyZS5ycGMuUGFzcxo5LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlN0c + mluZ1Jlc3BvbnNlIgASiAEKEFNlbmRFcnJvckNvbW1hbmQSNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlL + nJwYy5FcnJvckNvbW1hbmQaOS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5TdHJpbmdSZXNwb25zZ + SIAEoIBCg1TZW5kUmVjdXJzaW9uEjQuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuUmVjdXJzaW9uG + jkuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuU3RyaW5nUmVzcG9uc2UiABJ+CgtTZW5kQ29sbGVjd + BIyLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkNvbGxlY3QaOS5lZHUudWNpLmljcy5hbWJlci5lb + mdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5TdHJpbmdSZXNwb25zZSIAEokBChJTZW5kR2VuZXJhdGVOdW1iZXISOS5lZHUudWNpLmljc + y5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5HZW5lcmF0ZU51bWJlcho2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hc + mNoaXRlY3R1cmUucnBjLkludFJlc3BvbnNlIgASggEKDVNlbmRNdWx0aUNhbGwSNC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY + XJjaGl0ZWN0dXJlLnJwYy5NdWx0aUNhbGwaOS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5TdHJpb + mdSZXNwb25zZSIAEnoKCVNlbmRDaGFpbhIwLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkNoYWluG + jkuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuU3RyaW5nUmVzcG9uc2UiAEIL4j8IEAFIAFgAeAFiB + nByb3RvMw==""" + ).mkString) + lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { + val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) + _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) + } + lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { + val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) + com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( + edu.uci.ics.amber.engine.architecture.rpc.ControlCommandsProto.javaDescriptor, + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnsProto.javaDescriptor, + scalapb.options.ScalapbProto.javaDescriptor + )) + } + @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") + def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/TypedValue.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/TypedValue.scala similarity index 87% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/TypedValue.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/TypedValue.scala index 5a02a3f6b1f..dea23c1cee7 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/TypedValue.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/TypedValue.scala @@ -3,7 +3,7 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlreturns +package edu.uci.ics.amber.engine.architecture.rpc @SerialVersionUID(0L) final case class TypedValue( @@ -135,13 +135,13 @@ final case class TypedValue( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue.type = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue + def companion: edu.uci.ics.amber.engine.architecture.rpc.TypedValue.type = edu.uci.ics.amber.engine.architecture.rpc.TypedValue // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.TypedValue]) } -object TypedValue extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue = { +object TypedValue extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.TypedValue] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.TypedValue] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.TypedValue = { var __expression: _root_.scala.Predef.String = "" var __valueRef: _root_.scala.Predef.String = "" var __valueStr: _root_.scala.Predef.String = "" @@ -165,7 +165,7 @@ object TypedValue extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.en case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue( + edu.uci.ics.amber.engine.architecture.rpc.TypedValue( expression = __expression, valueRef = __valueRef, valueStr = __valueStr, @@ -173,10 +173,10 @@ object TypedValue extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.en expandable = __expandable ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.TypedValue] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue( + edu.uci.ics.amber.engine.architecture.rpc.TypedValue( expression = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), valueRef = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), valueStr = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), @@ -185,19 +185,19 @@ object TypedValue extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.en ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlreturnsProto.javaDescriptor.getMessageTypes().get(10) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlreturnsProto.scalaDescriptor.messages(10) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlReturnsProto.javaDescriptor.getMessageTypes().get(10) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlReturnsProto.scalaDescriptor.messages(10) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.TypedValue( expression = "", valueRef = "", valueStr = "", valueType = "", expandable = false ) - implicit class TypedValueLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue](_l) { + implicit class TypedValueLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.TypedValue]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.TypedValue](_l) { def expression: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.expression)((c_, f_) => c_.copy(expression = f_)) def valueRef: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.valueRef)((c_, f_) => c_.copy(valueRef = f_)) def valueStr: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.valueStr)((c_, f_) => c_.copy(valueStr = f_)) @@ -215,7 +215,7 @@ object TypedValue extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.en valueStr: _root_.scala.Predef.String, valueType: _root_.scala.Predef.String, expandable: _root_.scala.Boolean - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.TypedValue = _root_.edu.uci.ics.amber.engine.architecture.rpc.TypedValue( expression, valueRef, valueStr, diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkerServiceFs2Grpc.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkerServiceFs2Grpc.scala new file mode 100644 index 00000000000..7d21a76b1d3 --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkerServiceFs2Grpc.scala @@ -0,0 +1,138 @@ +package edu.uci.ics.amber.engine.architecture.rpc + +import _root_.cats.syntax.all._ + +trait WorkerServiceFs2Grpc[F[_], A] { + def addInputChannel(request: edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def addPartitioning(request: edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def assignPort(request: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def finalizeCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse] + def flushNetworkBuffer(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def initializeExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def openExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def pauseWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] + def prepareCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def queryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse] + def resumeWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] + def retrieveState(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def retryCurrentTuple(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def startWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] + def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] + def noOperation(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] +} + +object WorkerServiceFs2Grpc extends _root_.fs2.grpc.GeneratedCompanion[WorkerServiceFs2Grpc] { + + def mkClient[F[_]: _root_.cats.effect.Async, A](dispatcher: _root_.cats.effect.std.Dispatcher[F], channel: _root_.io.grpc.Channel, mkMetadata: A => F[_root_.io.grpc.Metadata], clientOptions: _root_.fs2.grpc.client.ClientOptions): WorkerServiceFs2Grpc[F, A] = new WorkerServiceFs2Grpc[F, A] { + def addInputChannel(request: edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_ADD_INPUT_CHANNEL, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def addPartitioning(request: edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_ADD_PARTITIONING, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def assignPort(request: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_ASSIGN_PORT, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def finalizeCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_FINALIZE_CHECKPOINT, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def flushNetworkBuffer(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_FLUSH_NETWORK_BUFFER, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def initializeExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_INITIALIZE_EXECUTOR, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def openExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_OPEN_EXECUTOR, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def pauseWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_PAUSE_WORKER, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def prepareCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_PREPARE_CHECKPOINT, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def queryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_QUERY_STATISTICS, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def resumeWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_RESUME_WORKER, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def retrieveState(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_RETRIEVE_STATE, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def retryCurrentTuple(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_RETRY_CURRENT_TUPLE, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def startWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_START_WORKER, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_DEBUG_COMMAND, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_EVALUATE_PYTHON_EXPRESSION, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + def noOperation(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { + mkMetadata(ctx).flatMap { m => + _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_NO_OPERATION, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) + } + } + } + + protected def serviceBinding[F[_]: _root_.cats.effect.Async, A](dispatcher: _root_.cats.effect.std.Dispatcher[F], serviceImpl: WorkerServiceFs2Grpc[F, A], mkCtx: _root_.io.grpc.Metadata => F[A], serverOptions: _root_.fs2.grpc.server.ServerOptions): _root_.io.grpc.ServerServiceDefinition = { + _root_.io.grpc.ServerServiceDefinition + .builder(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.SERVICE) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_ADD_INPUT_CHANNEL, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.addInputChannel(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_ADD_PARTITIONING, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.addPartitioning(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_ASSIGN_PORT, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.assignPort(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_FINALIZE_CHECKPOINT, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.finalizeCheckpoint(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_FLUSH_NETWORK_BUFFER, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.flushNetworkBuffer(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_INITIALIZE_EXECUTOR, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.initializeExecutor(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_OPEN_EXECUTOR, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.openExecutor(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_PAUSE_WORKER, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.pauseWorker(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_PREPARE_CHECKPOINT, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.prepareCheckpoint(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_QUERY_STATISTICS, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.queryStatistics(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_RESUME_WORKER, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.resumeWorker(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_RETRIEVE_STATE, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.retrieveState(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_RETRY_CURRENT_TUPLE, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.retryCurrentTuple(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_START_WORKER, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.startWorker(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_DEBUG_COMMAND, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.debugCommand(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_EVALUATE_PYTHON_EXPRESSION, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]((r, m) => mkCtx(m).flatMap(serviceImpl.evaluatePythonExpression(r, _)))) + .addMethod(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_NO_OPERATION, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.noOperation(r, _)))) + .build() + } + +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/workerservice/WorkerServiceGrpc.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkerServiceGrpc.scala similarity index 56% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/workerservice/WorkerServiceGrpc.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkerServiceGrpc.scala index 1076bfb081a..82bbd85bafb 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/workerservice/WorkerServiceGrpc.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkerServiceGrpc.scala @@ -3,183 +3,183 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.workerservice +package edu.uci.ics.amber.engine.architecture.rpc object WorkerServiceGrpc { - val METHOD_ADD_INPUT_CHANNEL: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_ADD_INPUT_CHANNEL: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "AddInputChannel")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(0))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(0))) .build() - val METHOD_ADD_PARTITIONING: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_ADD_PARTITIONING: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "AddPartitioning")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(1))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(1))) .build() - val METHOD_ASSIGN_PORT: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_ASSIGN_PORT: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "AssignPort")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(2))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(2))) .build() - val METHOD_FINALIZE_CHECKPOINT: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse] = + val METHOD_FINALIZE_CHECKPOINT: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "FinalizeCheckpoint")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(3))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(3))) .build() - val METHOD_FLUSH_NETWORK_BUFFER: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_FLUSH_NETWORK_BUFFER: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "FlushNetworkBuffer")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(4))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(4))) .build() - val METHOD_INITIALIZE_EXECUTOR: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_INITIALIZE_EXECUTOR: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "InitializeExecutor")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(5))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(5))) .build() - val METHOD_OPEN_EXECUTOR: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_OPEN_EXECUTOR: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "OpenExecutor")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(6))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(6))) .build() - val METHOD_PAUSE_WORKER: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = + val METHOD_PAUSE_WORKER: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "PauseWorker")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(7))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(7))) .build() - val METHOD_PREPARE_CHECKPOINT: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_PREPARE_CHECKPOINT: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "PrepareCheckpoint")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(8))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(8))) .build() - val METHOD_QUERY_STATISTICS: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse] = + val METHOD_QUERY_STATISTICS: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "QueryStatistics")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(9))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(9))) .build() - val METHOD_RESUME_WORKER: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = + val METHOD_RESUME_WORKER: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "ResumeWorker")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(10))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(10))) .build() - val METHOD_RETRIEVE_STATE: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_RETRIEVE_STATE: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "RetrieveState")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(11))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(11))) .build() - val METHOD_RETRY_CURRENT_TUPLE: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_RETRY_CURRENT_TUPLE: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "RetryCurrentTuple")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(12))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(12))) .build() - val METHOD_START_WORKER: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = + val METHOD_START_WORKER: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "StartWorker")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(13))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(13))) .build() - val METHOD_DEBUG_COMMAND: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_DEBUG_COMMAND: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "DebugCommand")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(14))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(14))) .build() - val METHOD_EVALUATE_PYTHON_EXPRESSION: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] = + val METHOD_EVALUATE_PYTHON_EXPRESSION: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "EvaluatePythonExpression")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(15))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(15))) .build() - val METHOD_NO_OPERATION: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = + val METHOD_NO_OPERATION: _root_.io.grpc.MethodDescriptor[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = _root_.io.grpc.MethodDescriptor.newBuilder() .setType(_root_.io.grpc.MethodDescriptor.MethodType.UNARY) .setFullMethodName(_root_.io.grpc.MethodDescriptor.generateFullMethodName("edu.uci.ics.amber.engine.architecture.rpc.WorkerService", "NoOperation")) .setSampledToLocalTracing(true) - .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest]) - .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]) - .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0).getMethods().get(16))) + .setRequestMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest]) + .setResponseMarshaller(_root_.scalapb.grpc.Marshaller.forMessage[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]) + .setSchemaDescriptor(_root_.scalapb.grpc.ConcreteProtoMethodDescriptorSupplier.fromMethodDescriptor(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0).getMethods().get(16))) .build() val SERVICE: _root_.io.grpc.ServiceDescriptor = _root_.io.grpc.ServiceDescriptor.newBuilder("edu.uci.ics.amber.engine.architecture.rpc.WorkerService") - .setSchemaDescriptor(new _root_.scalapb.grpc.ConcreteProtoFileDescriptorSupplier(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor)) + .setSchemaDescriptor(new _root_.scalapb.grpc.ConcreteProtoFileDescriptorSupplier(edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor)) .addMethod(METHOD_ADD_INPUT_CHANNEL) .addMethod(METHOD_ADD_PARTITIONING) .addMethod(METHOD_ASSIGN_PORT) @@ -203,147 +203,147 @@ object WorkerServiceGrpc { */ trait WorkerService extends _root_.scalapb.grpc.AbstractService { override def serviceCompanion = WorkerService - def addInputChannel(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def addPartitioning(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def assignPort(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def finalizeCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse] - def flushNetworkBuffer(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def initializeExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def openExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def pauseWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] - def prepareCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def queryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse] - def resumeWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] - def retrieveState(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def retryCurrentTuple(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def startWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] - def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] - def noOperation(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] + def addInputChannel(request: edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def addPartitioning(request: edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def assignPort(request: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def finalizeCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse] + def flushNetworkBuffer(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def initializeExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def openExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def pauseWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] + def prepareCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def queryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse] + def resumeWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] + def retrieveState(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def retryCurrentTuple(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def startWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] + def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] + def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] + def noOperation(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] } object WorkerService extends _root_.scalapb.grpc.ServiceCompanion[WorkerService] { implicit def serviceCompanion: _root_.scalapb.grpc.ServiceCompanion[WorkerService] = this - def javaDescriptor: _root_.com.google.protobuf.Descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.scalaDescriptor.services(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.scalaDescriptor.services(0) def bindService(serviceImpl: WorkerService, executionContext: scala.concurrent.ExecutionContext): _root_.io.grpc.ServerServiceDefinition = _root_.io.grpc.ServerServiceDefinition.builder(SERVICE) .addMethod( METHOD_ADD_INPUT_CHANNEL, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.addInputChannel(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_ADD_PARTITIONING, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.addPartitioning(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_ASSIGN_PORT, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.assignPort(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_FINALIZE_CHECKPOINT, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse]): _root_.scala.Unit = serviceImpl.finalizeCheckpoint(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_FLUSH_NETWORK_BUFFER, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.flushNetworkBuffer(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_INITIALIZE_EXECUTOR, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.initializeExecutor(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_OPEN_EXECUTOR, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.openExecutor(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_PAUSE_WORKER, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]): _root_.scala.Unit = serviceImpl.pauseWorker(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_PREPARE_CHECKPOINT, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.prepareCheckpoint(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_QUERY_STATISTICS, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse]): _root_.scala.Unit = serviceImpl.queryStatistics(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_RESUME_WORKER, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]): _root_.scala.Unit = serviceImpl.resumeWorker(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_RETRIEVE_STATE, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.retrieveState(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_RETRY_CURRENT_TUPLE, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.retryCurrentTuple(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_START_WORKER, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse]): _root_.scala.Unit = serviceImpl.startWorker(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_DEBUG_COMMAND, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.debugCommand(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_EVALUATE_PYTHON_EXPRESSION, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]): _root_.scala.Unit = serviceImpl.evaluatePythonExpression(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) .addMethod( METHOD_NO_OPERATION, - _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] { - override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]): _root_.scala.Unit = + _root_.io.grpc.stub.ServerCalls.asyncUnaryCall(new _root_.io.grpc.stub.ServerCalls.UnaryMethod[edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] { + override def invoke(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest, observer: _root_.io.grpc.stub.StreamObserver[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn]): _root_.scala.Unit = serviceImpl.noOperation(request).onComplete(scalapb.grpc.Grpc.completeObserver(observer))( executionContext) })) @@ -354,91 +354,91 @@ object WorkerServiceGrpc { */ trait WorkerServiceBlockingClient { def serviceCompanion = WorkerService - def addInputChannel(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def addPartitioning(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def assignPort(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def finalizeCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse - def flushNetworkBuffer(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def initializeExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def openExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def pauseWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse - def prepareCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def queryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse - def resumeWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse - def retrieveState(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def retryCurrentTuple(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def startWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse - def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn - def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue - def noOperation(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn + def addInputChannel(request: edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def addPartitioning(request: edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def assignPort(request: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def finalizeCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest): edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse + def flushNetworkBuffer(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def initializeExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def openExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def pauseWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse + def prepareCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def queryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse + def resumeWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse + def retrieveState(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def retryCurrentTuple(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def startWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse + def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn + def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest): edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue + def noOperation(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn } class WorkerServiceBlockingStub(channel: _root_.io.grpc.Channel, options: _root_.io.grpc.CallOptions = _root_.io.grpc.CallOptions.DEFAULT) extends _root_.io.grpc.stub.AbstractStub[WorkerServiceBlockingStub](channel, options) with WorkerServiceBlockingClient { - override def addInputChannel(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def addInputChannel(request: edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_ADD_INPUT_CHANNEL, options, request) } - override def addPartitioning(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def addPartitioning(request: edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_ADD_PARTITIONING, options, request) } - override def assignPort(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def assignPort(request: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_ASSIGN_PORT, options, request) } - override def finalizeCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse = { + override def finalizeCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest): edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_FINALIZE_CHECKPOINT, options, request) } - override def flushNetworkBuffer(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def flushNetworkBuffer(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_FLUSH_NETWORK_BUFFER, options, request) } - override def initializeExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def initializeExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_INITIALIZE_EXECUTOR, options, request) } - override def openExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def openExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_OPEN_EXECUTOR, options, request) } - override def pauseWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse = { + override def pauseWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_PAUSE_WORKER, options, request) } - override def prepareCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def prepareCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_PREPARE_CHECKPOINT, options, request) } - override def queryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse = { + override def queryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_QUERY_STATISTICS, options, request) } - override def resumeWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse = { + override def resumeWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_RESUME_WORKER, options, request) } - override def retrieveState(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def retrieveState(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_RETRIEVE_STATE, options, request) } - override def retryCurrentTuple(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def retryCurrentTuple(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_RETRY_CURRENT_TUPLE, options, request) } - override def startWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse = { + override def startWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_START_WORKER, options, request) } - override def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_DEBUG_COMMAND, options, request) } - override def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue = { + override def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest): edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_EVALUATE_PYTHON_EXPRESSION, options, request) } - override def noOperation(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn = { + override def noOperation(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn = { _root_.scalapb.grpc.ClientCalls.blockingUnaryCall(channel, METHOD_NO_OPERATION, options, request) } @@ -446,71 +446,71 @@ object WorkerServiceGrpc { } class WorkerServiceStub(channel: _root_.io.grpc.Channel, options: _root_.io.grpc.CallOptions = _root_.io.grpc.CallOptions.DEFAULT) extends _root_.io.grpc.stub.AbstractStub[WorkerServiceStub](channel, options) with WorkerService { - override def addInputChannel(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def addInputChannel(request: edu.uci.ics.amber.engine.architecture.rpc.AddInputChannelRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_ADD_INPUT_CHANNEL, options, request) } - override def addPartitioning(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def addPartitioning(request: edu.uci.ics.amber.engine.architecture.rpc.AddPartitioningRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_ADD_PARTITIONING, options, request) } - override def assignPort(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def assignPort(request: edu.uci.ics.amber.engine.architecture.rpc.AssignPortRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_ASSIGN_PORT, options, request) } - override def finalizeCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse] = { + override def finalizeCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_FINALIZE_CHECKPOINT, options, request) } - override def flushNetworkBuffer(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def flushNetworkBuffer(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_FLUSH_NETWORK_BUFFER, options, request) } - override def initializeExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def initializeExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.InitializeExecutorRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_INITIALIZE_EXECUTOR, options, request) } - override def openExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def openExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_OPEN_EXECUTOR, options, request) } - override def pauseWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = { + override def pauseWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_PAUSE_WORKER, options, request) } - override def prepareCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def prepareCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.PrepareCheckpointRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_PREPARE_CHECKPOINT, options, request) } - override def queryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse] = { + override def queryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.WorkerMetricsResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_QUERY_STATISTICS, options, request) } - override def resumeWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = { + override def resumeWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_RESUME_WORKER, options, request) } - override def retrieveState(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def retrieveState(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_RETRIEVE_STATE, options, request) } - override def retryCurrentTuple(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def retryCurrentTuple(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_RETRY_CURRENT_TUPLE, options, request) } - override def startWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = { + override def startWorker(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.WorkerStateResponse] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_START_WORKER, options, request) } - override def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.DebugCommandRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_DEBUG_COMMAND, options, request) } - override def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] = { + override def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_EVALUATE_PYTHON_EXPRESSION, options, request) } - override def noOperation(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { + override def noOperation(request: edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest): scala.concurrent.Future[edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn] = { _root_.scalapb.grpc.ClientCalls.asyncUnaryCall(channel, METHOD_NO_OPERATION, options, request) } @@ -529,6 +529,6 @@ object WorkerServiceGrpc { def stub(channel: _root_.io.grpc.Channel): WorkerServiceStub = new WorkerServiceStub(channel) - def javaDescriptor: _root_.com.google.protobuf.Descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerserviceProto.javaDescriptor.getServices().get(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.ServiceDescriptor = edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceProto.javaDescriptor.getServices().get(0) } \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkerServiceProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkerServiceProto.scala new file mode 100644 index 00000000000..460b33d20bd --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkerServiceProto.scala @@ -0,0 +1,68 @@ +// Generated by the Scala Plugin for the Protocol Buffer Compiler. +// Do not edit! +// +// Protofile syntax: PROTO3 + +package edu.uci.ics.amber.engine.architecture.rpc + +object WorkerServiceProto extends _root_.scalapb.GeneratedFileObject { + lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( + edu.uci.ics.amber.engine.architecture.rpc.ControlCommandsProto, + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnsProto, + scalapb.options.ScalapbProto + ) + lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty + private lazy val ProtoBytes: _root_.scala.Array[Byte] = + scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( + """Cj5lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy93b3JrZXJfc2VydmljZS5wcm90bxIpZWR1LnVja + S5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMaQGVkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9hcmNoaXRlY3R1cmUvc + nBjL2NvbnRyb2xfY29tbWFuZHMucHJvdG8aP2VkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9hcmNoaXRlY3R1cmUvcnBjL2NvbnRyb + 2xfcmV0dXJucy5wcm90bxoVc2NhbGFwYi9zY2FsYXBiLnByb3RvMukSCg1Xb3JrZXJTZXJ2aWNlEowBCg9BZGRJbnB1dENoYW5uZ + WwSQS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5BZGRJbnB1dENoYW5uZWxSZXF1ZXN0GjYuZWR1L + nVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm4SjAEKD0FkZFBhcnRpdGlvbmluZxJBLmVkd + S51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkFkZFBhcnRpdGlvbmluZ1JlcXVlc3QaNi5lZHUudWNpLmljc + y5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJldHVybhKCAQoKQXNzaWduUG9ydBI8LmVkdS51Y2kuaWNzLmFtY + mVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkFzc2lnblBvcnRSZXF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY + 2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm4SoQEKEkZpbmFsaXplQ2hlY2twb2ludBJELmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZ + S5hcmNoaXRlY3R1cmUucnBjLkZpbmFsaXplQ2hlY2twb2ludFJlcXVlc3QaRS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJja + Gl0ZWN0dXJlLnJwYy5GaW5hbGl6ZUNoZWNrcG9pbnRSZXNwb25zZRKFAQoSRmx1c2hOZXR3b3JrQnVmZmVyEjcuZWR1LnVjaS5pY + 3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY + 2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm4SkgEKEkluaXRpYWxpemVFeGVjdXRvchJELmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZ + S5hcmNoaXRlY3R1cmUucnBjLkluaXRpYWxpemVFeGVjdXRvclJlcXVlc3QaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJja + Gl0ZWN0dXJlLnJwYy5FbXB0eVJldHVybhJ/CgxPcGVuRXhlY3V0b3ISNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0Z + WN0dXJlLnJwYy5FbXB0eVJlcXVlc3QaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJld + HVybhKGAQoLUGF1c2VXb3JrZXISNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJlcXVlc + 3QaPi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Xb3JrZXJTdGF0ZVJlc3BvbnNlEpABChFQcmVwY + XJlQ2hlY2twb2ludBJDLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlByZXBhcmVDaGVja3BvaW50U + mVxdWVzdBo2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmV0dXJuEowBCg9RdWVyeVN0Y + XRpc3RpY3MSNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJlcXVlc3QaQC5lZHUudWNpL + mljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Xb3JrZXJNZXRyaWNzUmVzcG9uc2UShwEKDFJlc3VtZVdvcmtlchI3L + mVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmVxdWVzdBo+LmVkdS51Y2kuaWNzLmFtYmVyL + mVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLldvcmtlclN0YXRlUmVzcG9uc2USgAEKDVJldHJpZXZlU3RhdGUSNy5lZHUudWNpLmljc + y5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJlcXVlc3QaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJja + Gl0ZWN0dXJlLnJwYy5FbXB0eVJldHVybhKEAQoRUmV0cnlDdXJyZW50VHVwbGUSNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY + XJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJlcXVlc3QaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Fb + XB0eVJldHVybhKGAQoLU3RhcnRXb3JrZXISNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0e + VJlcXVlc3QaPi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Xb3JrZXJTdGF0ZVJlc3BvbnNlEoYBC + gxEZWJ1Z0NvbW1hbmQSPi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5EZWJ1Z0NvbW1hbmRSZXF1Z + XN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm4SoQEKGEV2YWx1YXRlUHl0a + G9uRXhwcmVzc2lvbhJKLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkV2YWx1YXRlUHl0aG9uRXhwc + mVzc2lvblJlcXVlc3QaOS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FdmFsdWF0ZWRWYWx1ZRJ+C + gtOb09wZXJhdGlvbhI3LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmVxdWVzdBo2LmVkd + S51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmV0dXJuQgviPwgQAUgAWAB4AWIGcHJvdG8z""" + ).mkString) + lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { + val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) + _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) + } + lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { + val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) + com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( + edu.uci.ics.amber.engine.architecture.rpc.ControlCommandsProto.javaDescriptor, + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnsProto.javaDescriptor, + scalapb.options.ScalapbProto.javaDescriptor + )) + } + @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") + def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/WorkflowReconfigureRequest.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkflowReconfigureRequest.scala similarity index 69% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/WorkflowReconfigureRequest.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkflowReconfigureRequest.scala index 0bbe81bb3fa..77582d15b3c 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/WorkflowReconfigureRequest.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/WorkflowReconfigureRequest.scala @@ -3,11 +3,11 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlcommands +package edu.uci.ics.amber.engine.architecture.rpc @SerialVersionUID(0L) final case class WorkflowReconfigureRequest( - reconfiguration: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest, + reconfiguration: edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest, reconfigurationId: _root_.scala.Predef.String ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[WorkflowReconfigureRequest] { @transient @@ -55,13 +55,13 @@ final case class WorkflowReconfigureRequest( } }; } - def withReconfiguration(__v: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest): WorkflowReconfigureRequest = copy(reconfiguration = __v) + def withReconfiguration(__v: edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest): WorkflowReconfigureRequest = copy(reconfiguration = __v) def withReconfigurationId(__v: _root_.scala.Predef.String): WorkflowReconfigureRequest = copy(reconfigurationId = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = reconfiguration - if (__t != edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest.defaultInstance) __t else null } case 2 => { val __t = reconfigurationId @@ -77,14 +77,14 @@ final case class WorkflowReconfigureRequest( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest.type = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest + def companion: edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest.type = edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest]) } -object WorkflowReconfigureRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest = { - var __reconfiguration: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest] = _root_.scala.None +object WorkflowReconfigureRequest extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest = { + var __reconfiguration: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest] = _root_.scala.None var __reconfigurationId: _root_.scala.Predef.String = "" var _done__ = false while (!_done__) { @@ -92,51 +92,51 @@ object WorkflowReconfigureRequest extends scalapb.GeneratedMessageCompanion[edu. _tag__ match { case 0 => _done__ = true case 10 => - __reconfiguration = _root_.scala.Some(__reconfiguration.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __reconfiguration = _root_.scala.Some(__reconfiguration.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => __reconfigurationId = _input__.readStringRequireUtf8() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest( - reconfiguration = __reconfiguration.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest( + reconfiguration = __reconfiguration.getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest.defaultInstance), reconfigurationId = __reconfigurationId ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest( - reconfiguration = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest.defaultInstance), + edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest( + reconfiguration = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest.defaultInstance), reconfigurationId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlcommandsProto.javaDescriptor.getMessageTypes().get(7) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlcommandsProto.scalaDescriptor.messages(7) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ControlCommandsProto.javaDescriptor.getMessageTypes().get(7) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ControlCommandsProto.scalaDescriptor.messages(7) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest + case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest( - reconfiguration = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest.defaultInstance, + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest( + reconfiguration = edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest.defaultInstance, reconfigurationId = "" ) - implicit class WorkflowReconfigureRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest](_l) { - def reconfiguration: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest] = field(_.reconfiguration)((c_, f_) => c_.copy(reconfiguration = f_)) + implicit class WorkflowReconfigureRequestLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest](_l) { + def reconfiguration: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest] = field(_.reconfiguration)((c_, f_) => c_.copy(reconfiguration = f_)) def reconfigurationId: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.reconfigurationId)((c_, f_) => c_.copy(reconfigurationId = f_)) } final val RECONFIGURATION_FIELD_NUMBER = 1 - final val RECONFIGURATIONID_FIELD_NUMBER = 2 + final val RECONFIGURATION_ID_FIELD_NUMBER = 2 def of( - reconfiguration: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest, + reconfiguration: edu.uci.ics.amber.engine.architecture.rpc.ModifyLogicRequest, reconfigurationId: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest( + ): _root_.edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest = _root_.edu.uci.ics.amber.engine.architecture.rpc.WorkflowReconfigureRequest( reconfiguration, reconfigurationId ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ControlcommandsProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ControlcommandsProto.scala deleted file mode 100644 index 16776aba2b6..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlcommands/ControlcommandsProto.scala +++ /dev/null @@ -1,227 +0,0 @@ -// Generated by the Scala Plugin for the Protocol Buffer Compiler. -// Do not edit! -// -// Protofile syntax: PROTO3 - -package edu.uci.ics.amber.engine.architecture.rpc.controlcommands - -object ControlcommandsProto extends _root_.scalapb.GeneratedFileObject { - lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( - edu.uci.ics.amber.engine.common.virtualidentity.VirtualidentityProto, - edu.uci.ics.amber.engine.common.workflow.WorkflowProto, - edu.uci.ics.amber.engine.architecture.worker.statistics.StatisticsProto, - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningsProto, - scalapb.options.ScalapbProto, - com.google.protobuf.timestamp.TimestampProto, - com.google.protobuf.any.AnyProto - ) - lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = - Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlRequestMessage, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ChannelMarkerPayload, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkflowReconfigureRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ModifyLogicRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.UpdateExecutorRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest, - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest - ) - private lazy val ProtoBytes: _root_.scala.Array[Byte] = - scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( - """Cj9lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy9jb250cm9sY29tbWFuZHMucHJvdG8SKWVkdS51Y - 2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjGjVlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3ZpcnR1Y - WxpZGVudGl0eS5wcm90bxouZWR1L3VjaS9pY3MvYW1iZXIvZW5naW5lL2NvbW1vbi93b3JrZmxvdy5wcm90bxo9ZWR1L3VjaS9pY - 3MvYW1iZXIvZW5naW5lL2FyY2hpdGVjdHVyZS93b3JrZXIvc3RhdGlzdGljcy5wcm90bxpHZWR1L3VjaS9pY3MvYW1iZXIvZW5na - W5lL2FyY2hpdGVjdHVyZS9zZW5kc2VtYW50aWNzL3BhcnRpdGlvbmluZ3MucHJvdG8aFXNjYWxhcGIvc2NhbGFwYi5wcm90bxofZ - 29vZ2xlL3Byb3RvYnVmL3RpbWVzdGFtcC5wcm90bxoZZ29vZ2xlL3Byb3RvYnVmL2FueS5wcm90byKAHwoOQ29udHJvbFJlcXVlc - 3QStAEKHXByb3BhZ2F0ZUNoYW5uZWxNYXJrZXJSZXF1ZXN0GAEgASgLMkguZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpd - GVjdHVyZS5ycGMuUHJvcGFnYXRlQ2hhbm5lbE1hcmtlclJlcXVlc3RCIuI/HxIdcHJvcGFnYXRlQ2hhbm5lbE1hcmtlclJlcXVlc - 3RIAFIdcHJvcGFnYXRlQ2hhbm5lbE1hcmtlclJlcXVlc3QSrAEKG3Rha2VHbG9iYWxDaGVja3BvaW50UmVxdWVzdBgCIAEoCzJGL - mVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlRha2VHbG9iYWxDaGVja3BvaW50UmVxdWVzdEIg4j8dE - ht0YWtlR2xvYmFsQ2hlY2twb2ludFJlcXVlc3RIAFIbdGFrZUdsb2JhbENoZWNrcG9pbnRSZXF1ZXN0EowBChNkZWJ1Z0NvbW1hb - mRSZXF1ZXN0GAMgASgLMj4uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRGVidWdDb21tYW5kUmVxd - WVzdEIY4j8VEhNkZWJ1Z0NvbW1hbmRSZXF1ZXN0SABSE2RlYnVnQ29tbWFuZFJlcXVlc3QSvAEKH2V2YWx1YXRlUHl0aG9uRXhwc - mVzc2lvblJlcXVlc3QYBCABKAsySi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FdmFsdWF0ZVB5d - GhvbkV4cHJlc3Npb25SZXF1ZXN0QiTiPyESH2V2YWx1YXRlUHl0aG9uRXhwcmVzc2lvblJlcXVlc3RIAFIfZXZhbHVhdGVQeXRob - 25FeHByZXNzaW9uUmVxdWVzdBKIAQoSbW9kaWZ5TG9naWNSZXF1ZXN0GAUgASgLMj0uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lL - mFyY2hpdGVjdHVyZS5ycGMuTW9kaWZ5TG9naWNSZXF1ZXN0QhfiPxQSEm1vZGlmeUxvZ2ljUmVxdWVzdEgAUhJtb2RpZnlMb2dpY - 1JlcXVlc3QSkAEKFHJldHJ5V29ya2Zsb3dSZXF1ZXN0GAYgASgLMj8uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjd - HVyZS5ycGMuUmV0cnlXb3JrZmxvd1JlcXVlc3RCGeI/FhIUcmV0cnlXb3JrZmxvd1JlcXVlc3RIAFIUcmV0cnlXb3JrZmxvd1Jlc - XVlc3QSuAEKHmNvbnNvbGVNZXNzYWdlVHJpZ2dlcmVkUmVxdWVzdBgIIAEoCzJJLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hc - mNoaXRlY3R1cmUucnBjLkNvbnNvbGVNZXNzYWdlVHJpZ2dlcmVkUmVxdWVzdEIj4j8gEh5jb25zb2xlTWVzc2FnZVRyaWdnZXJlZ - FJlcXVlc3RIAFIeY29uc29sZU1lc3NhZ2VUcmlnZ2VyZWRSZXF1ZXN0EpABChRwb3J0Q29tcGxldGVkUmVxdWVzdBgJIAEoCzI/L - mVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlBvcnRDb21wbGV0ZWRSZXF1ZXN0QhniPxYSFHBvcnRDb - 21wbGV0ZWRSZXF1ZXN0SABSFHBvcnRDb21wbGV0ZWRSZXF1ZXN0EqQBChl3b3JrZXJTdGF0ZVVwZGF0ZWRSZXF1ZXN0GAogASgLM - kQuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuV29ya2VyU3RhdGVVcGRhdGVkUmVxdWVzdEIe4j8bE - hl3b3JrZXJTdGF0ZVVwZGF0ZWRSZXF1ZXN0SABSGXdvcmtlclN0YXRlVXBkYXRlZFJlcXVlc3QSiAEKEmxpbmtXb3JrZXJzUmVxd - WVzdBgLIAEoCzI9LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkxpbmtXb3JrZXJzUmVxdWVzdEIX4 - j8UEhJsaW5rV29ya2Vyc1JlcXVlc3RIAFISbGlua1dvcmtlcnNSZXF1ZXN0EpgBChZhZGRJbnB1dENoYW5uZWxSZXF1ZXN0GDIgA - SgLMkEuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuQWRkSW5wdXRDaGFubmVsUmVxdWVzdEIb4j8YE - hZhZGRJbnB1dENoYW5uZWxSZXF1ZXN0SABSFmFkZElucHV0Q2hhbm5lbFJlcXVlc3QSmAEKFmFkZFBhcnRpdGlvbmluZ1JlcXVlc - 3QYMyABKAsyQS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5BZGRQYXJ0aXRpb25pbmdSZXF1ZXN0Q - hviPxgSFmFkZFBhcnRpdGlvbmluZ1JlcXVlc3RIAFIWYWRkUGFydGl0aW9uaW5nUmVxdWVzdBKEAQoRYXNzaWduUG9ydFJlcXVlc - 3QYNCABKAsyPC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Bc3NpZ25Qb3J0UmVxdWVzdEIW4j8TE - hFhc3NpZ25Qb3J0UmVxdWVzdEgAUhFhc3NpZ25Qb3J0UmVxdWVzdBKkAQoZZmluYWxpemVDaGVja3BvaW50UmVxdWVzdBg1IAEoC - zJELmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkZpbmFsaXplQ2hlY2twb2ludFJlcXVlc3RCHuI/G - xIZZmluYWxpemVDaGVja3BvaW50UmVxdWVzdEgAUhlmaW5hbGl6ZUNoZWNrcG9pbnRSZXF1ZXN0EqQBChlpbml0aWFsaXplRXhlY - 3V0b3JSZXF1ZXN0GDYgASgLMkQuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuSW5pdGlhbGl6ZUV4Z - WN1dG9yUmVxdWVzdEIe4j8bEhlpbml0aWFsaXplRXhlY3V0b3JSZXF1ZXN0SABSGWluaXRpYWxpemVFeGVjdXRvclJlcXVlc3QSl - AEKFXVwZGF0ZUV4ZWN1dG9yUmVxdWVzdBg3IAEoCzJALmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjL - lVwZGF0ZUV4ZWN1dG9yUmVxdWVzdEIa4j8XEhV1cGRhdGVFeGVjdXRvclJlcXVlc3RIAFIVdXBkYXRlRXhlY3V0b3JSZXF1ZXN0E - nAKDGVtcHR5UmVxdWVzdBg4IAEoCzI3LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmVxd - WVzdEIR4j8OEgxlbXB0eVJlcXVlc3RIAFIMZW1wdHlSZXF1ZXN0EqABChhwcmVwYXJlQ2hlY2twb2ludFJlcXVlc3QYOSABKAsyQ - y5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5QcmVwYXJlQ2hlY2twb2ludFJlcXVlc3RCHeI/GhIYc - HJlcGFyZUNoZWNrcG9pbnRSZXF1ZXN0SABSGHByZXBhcmVDaGVja3BvaW50UmVxdWVzdBKYAQoWcXVlcnlTdGF0aXN0aWNzUmVxd - WVzdBg6IAEoCzJBLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlF1ZXJ5U3RhdGlzdGljc1JlcXVlc - 3RCG+I/GBIWcXVlcnlTdGF0aXN0aWNzUmVxdWVzdEgAUhZxdWVyeVN0YXRpc3RpY3NSZXF1ZXN0ElAKBHBpbmcYZCABKAsyLy5lZ - HUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5QaW5nQgniPwYSBHBpbmdIAFIEcGluZxJQCgRwb25nGGUgA - SgLMi8uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuUG9uZ0IJ4j8GEgRwb25nSABSBHBvbmcSWAoGb - mVzdGVkGGYgASgLMjEuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuTmVzdGVkQgviPwgSBm5lc3RlZ - EgAUgZuZXN0ZWQSUAoEcGFzcxhnIAEoCzIvLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlBhc3NCC - eI/BhIEcGFzc0gAUgRwYXNzEnAKDGVycm9yQ29tbWFuZBhoIAEoCzI3LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY - 3R1cmUucnBjLkVycm9yQ29tbWFuZEIR4j8OEgxlcnJvckNvbW1hbmRIAFIMZXJyb3JDb21tYW5kEmQKCXJlY3Vyc2lvbhhpIAEoC - zI0LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlJlY3Vyc2lvbkIO4j8LEglyZWN1cnNpb25IAFIJc - mVjdXJzaW9uElwKB2NvbGxlY3QYaiABKAsyMi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Db2xsZ - WN0QgziPwkSB2NvbGxlY3RIAFIHY29sbGVjdBJ4Cg5nZW5lcmF0ZU51bWJlchhrIAEoCzI5LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ - 2luZS5hcmNoaXRlY3R1cmUucnBjLkdlbmVyYXRlTnVtYmVyQhPiPxASDmdlbmVyYXRlTnVtYmVySABSDmdlbmVyYXRlTnVtYmVyE - mQKCW11bHRpQ2FsbBhsIAEoCzI0LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLk11bHRpQ2FsbEIO4 - j8LEgltdWx0aUNhbGxIAFIJbXVsdGlDYWxsElQKBWNoYWluGG0gASgLMjAuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpd - GVjdHVyZS5ycGMuQ2hhaW5CCuI/BxIFY2hhaW5IAFIFY2hhaW5CDgoMc2VhbGVkX3ZhbHVlIg4KDEVtcHR5UmVxdWVzdCLcAQoPQ - XN5bmNSUENDb250ZXh0El0KBnNlbmRlchgBIAEoCzI1LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQWN0b3JWaXJ0d - WFsSWRlbnRpdHlCDuI/CxIGc2VuZGVy8AEBUgZzZW5kZXISYwoIcmVjZWl2ZXIYAiABKAsyNS5lZHUudWNpLmljcy5hbWJlci5lb - mdpbmUuY29tbW9uLkFjdG9yVmlydHVhbElkZW50aXR5QhDiPw0SCHJlY2VpdmVy8AEBUghyZWNlaXZlcjoF4j8COAEi/gIKEUNvb - nRyb2xJbnZvY2F0aW9uEi8KCm1ldGhvZE5hbWUYASABKAlCD+I/DBIKbWV0aG9kTmFtZVIKbWV0aG9kTmFtZRJkCgdjb21tYW5kG - AIgASgLMjkuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuQ29udHJvbFJlcXVlc3RCD+I/DBIHY29tb - WFuZPABAVIHY29tbWFuZBJiCgdjb250ZXh0GAMgASgLMjouZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5yc - GMuQXN5bmNSUENDb250ZXh0QgziPwkSB2NvbnRleHRSB2NvbnRleHQSLAoJY29tbWFuZElkGAQgASgDQg7iPwsSCWNvbW1hbmRJZ - FIJY29tbWFuZElkOkDiPz0KO2VkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uYW1iZXJtZXNzYWdlLkNvbnRyb2xQYXlsb - 2FkIqYFChRDaGFubmVsTWFya2VyUGF5bG9hZBJSCgJpZBgBIAEoCzI2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQ - 2hhbm5lbE1hcmtlcklkZW50aXR5QgriPwcSAmlk8AEBUgJpZBJtCgptYXJrZXJUeXBlGAIgASgOMjwuZWR1LnVjaS5pY3MuYW1iZ - XIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuQ2hhbm5lbE1hcmtlclR5cGVCD+I/DBIKbWFya2VyVHlwZVIKbWFya2VyVHlwZRJSC - gVzY29wZRgDIAMoCzIwLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQ2hhbm5lbElkZW50aXR5QgriPwcSBXNjb3BlU - gVzY29wZRKQAQoOY29tbWFuZE1hcHBpbmcYBCADKAsyUy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwY - y5DaGFubmVsTWFya2VyUGF5bG9hZC5Db21tYW5kTWFwcGluZ0VudHJ5QhPiPxASDmNvbW1hbmRNYXBwaW5nUg5jb21tYW5kTWFwc - GluZxqVAQoTQ29tbWFuZE1hcHBpbmdFbnRyeRIaCgNrZXkYASABKAlCCOI/BRIDa2V5UgNrZXkSXgoFdmFsdWUYAiABKAsyPC5lZ - HUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Db250cm9sSW52b2NhdGlvbkIK4j8HEgV2YWx1ZVIFdmFsd - WU6AjgBOkziP0kKR2VkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uYW1iZXJtZXNzYWdlLldvcmtmbG93RklGT01lc3NhZ - 2VQYXlsb2FkItUFCh1Qcm9wYWdhdGVDaGFubmVsTWFya2VyUmVxdWVzdBJ/ChNzb3VyY2VPcFRvU3RhcnRQcm9wGAEgAygLMjMuZ - WR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5QaHlzaWNhbE9wSWRlbnRpdHlCGOI/FRITc291cmNlT3BUb1N0YXJ0UHJvc - FITc291cmNlT3BUb1N0YXJ0UHJvcBJSCgJpZBgCIAEoCzI2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQ2hhbm5lb - E1hcmtlcklkZW50aXR5QgriPwcSAmlk8AEBUgJpZBJtCgptYXJrZXJUeXBlGAMgASgOMjwuZWR1LnVjaS5pY3MuYW1iZXIuZW5na - W5lLmFyY2hpdGVjdHVyZS5ycGMuQ2hhbm5lbE1hcmtlclR5cGVCD+I/DBIKbWFya2VyVHlwZVIKbWFya2VyVHlwZRJVCgVzY29wZ - RgEIAMoCzIzLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uUGh5c2ljYWxPcElkZW50aXR5QgriPwcSBXNjb3BlUgVzY - 29wZRJhCgl0YXJnZXRPcHMYBSADKAsyMy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLlBoeXNpY2FsT3BJZGVudGl0e - UIO4j8LEgl0YXJnZXRPcHNSCXRhcmdldE9wcxJzCg1tYXJrZXJDb21tYW5kGAYgASgLMjkuZWR1LnVjaS5pY3MuYW1iZXIuZW5na - W5lLmFyY2hpdGVjdHVyZS5ycGMuQ29udHJvbFJlcXVlc3RCEuI/DxINbWFya2VyQ29tbWFuZFINbWFya2VyQ29tbWFuZBJBChBtY - XJrZXJNZXRob2ROYW1lGAcgASgJQhXiPxISEG1hcmtlck1ldGhvZE5hbWVSEG1hcmtlck1ldGhvZE5hbWUigAIKG1Rha2VHbG9iY - WxDaGVja3BvaW50UmVxdWVzdBI7Cg5lc3RpbWF0aW9uT25seRgBIAEoCEIT4j8QEg5lc3RpbWF0aW9uT25seVIOZXN0aW1hdGlvb - k9ubHkScAoMY2hlY2twb2ludElkGAIgASgLMjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5DaGFubmVsTWFya2VyS - WRlbnRpdHlCFOI/ERIMY2hlY2twb2ludElk8AEBUgxjaGVja3BvaW50SWQSMgoLZGVzdGluYXRpb24YAyABKAlCEOI/DRILZGVzd - GluYXRpb25SC2Rlc3RpbmF0aW9uIuUBChpXb3JrZmxvd1JlY29uZmlndXJlUmVxdWVzdBKAAQoPcmVjb25maWd1cmF0aW9uGAEgA - SgLMj0uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuTW9kaWZ5TG9naWNSZXF1ZXN0QhfiPxQSD3JlY - 29uZmlndXJhdGlvbvABAVIPcmVjb25maWd1cmF0aW9uEkQKEXJlY29uZmlndXJhdGlvbklkGAIgASgJQhbiPxMSEXJlY29uZmlnd - XJhdGlvbklkUhFyZWNvbmZpZ3VyYXRpb25JZCJcChNEZWJ1Z0NvbW1hbmRSZXF1ZXN0EikKCHdvcmtlcklkGAEgASgJQg3iPwoSC - HdvcmtlcklkUgh3b3JrZXJJZBIaCgNjbWQYAiABKAlCCOI/BRIDY21kUgNjbWQigwEKH0V2YWx1YXRlUHl0aG9uRXhwcmVzc2lvb - lJlcXVlc3QSLwoKZXhwcmVzc2lvbhgBIAEoCUIP4j8MEgpleHByZXNzaW9uUgpleHByZXNzaW9uEi8KCm9wZXJhdG9ySWQYAiABK - AlCD+I/DBIKb3BlcmF0b3JJZFIKb3BlcmF0b3JJZCKQAQoSTW9kaWZ5TG9naWNSZXF1ZXN0EnoKDXVwZGF0ZVJlcXVlc3QYASADK - AsyQC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5VcGRhdGVFeGVjdXRvclJlcXVlc3RCEuI/DxINd - XBkYXRlUmVxdWVzdFINdXBkYXRlUmVxdWVzdCJ1ChRSZXRyeVdvcmtmbG93UmVxdWVzdBJdCgd3b3JrZXJzGAEgAygLMjUuZWR1L - nVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5BY3RvclZpcnR1YWxJZGVudGl0eUIM4j8JEgd3b3JrZXJzUgd3b3JrZXJzIqMDC - g5Db25zb2xlTWVzc2FnZRIqCgl3b3JrZXJfaWQYASABKAlCDeI/ChIId29ya2VySWRSCHdvcmtlcklkEksKCXRpbWVzdGFtcBgCI - AEoCzIaLmdvb2dsZS5wcm90b2J1Zi5UaW1lc3RhbXBCEeI/DhIJdGltZXN0YW1w8AEBUgl0aW1lc3RhbXASZgoIbXNnX3R5cGUYA - yABKA4yPS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Db25zb2xlTWVzc2FnZVR5cGVCDOI/CRIHb - XNnVHlwZVIHbXNnVHlwZRIjCgZzb3VyY2UYBCABKAlCC+I/CBIGc291cmNlUgZzb3VyY2USIAoFdGl0bGUYBSABKAlCCuI/BxIFd - Gl0bGVSBXRpdGxlEiYKB21lc3NhZ2UYBiABKAlCDOI/CRIHbWVzc2FnZVIHbWVzc2FnZTpB4j8+CjxlZHUudWNpLmljcy5hbWJlc - i5lbmdpbmUuYXJjaGl0ZWN0dXJlLmNvbnRyb2xsZXIuQ2xpZW50RXZlbnQimwEKHkNvbnNvbGVNZXNzYWdlVHJpZ2dlcmVkUmVxd - WVzdBJ5Cg5jb25zb2xlTWVzc2FnZRgBIAEoCzI5LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkNvb - nNvbGVNZXNzYWdlQhbiPxMSDmNvbnNvbGVNZXNzYWdl8AEBUg5jb25zb2xlTWVzc2FnZSKPAQoUUG9ydENvbXBsZXRlZFJlcXVlc - 3QSVQoGcG9ydElkGAEgASgLMi0uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5Qb3J0SWRlbnRpdHlCDuI/CxIGcG9yd - Elk8AEBUgZwb3J0SWQSIAoFaW5wdXQYAiABKAhCCuI/BxIFaW5wdXRSBWlucHV0InsKGVdvcmtlclN0YXRlVXBkYXRlZFJlcXVlc - 3QSXgoFc3RhdGUYASABKA4yOS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLndvcmtlci5Xb3JrZXJTdGF0Z - UIN4j8KEgVzdGF0ZfABAVIFc3RhdGUiZQoSTGlua1dvcmtlcnNSZXF1ZXN0Ek8KBGxpbmsYASABKAsyLS5lZHUudWNpLmljcy5hb - WJlci5lbmdpbmUuY29tbW9uLlBoeXNpY2FsTGlua0IM4j8JEgRsaW5r8AEBUgRsaW5rIosBCgRQaW5nEhQKAWkYASABKAVCBuI/A - xIBaVIBaRIaCgNlbmQYAiABKAVCCOI/BRIDZW5kUgNlbmQSUQoCdG8YAyABKAsyNS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY - 29tbW9uLkFjdG9yVmlydHVhbElkZW50aXR5QgriPwcSAnRv8AEBUgJ0byKLAQoEUG9uZxIUCgFpGAEgASgFQgbiPwMSAWlSAWkSG - goDZW5kGAIgASgFQgjiPwUSA2VuZFIDZW5kElEKAnRvGAMgASgLMjUuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5BY - 3RvclZpcnR1YWxJZGVudGl0eUIK4j8HEgJ0b/ABAVICdG8iKAoEUGFzcxIgCgV2YWx1ZRgBIAEoCUIK4j8HEgV2YWx1ZVIFdmFsd - WUiHgoGTmVzdGVkEhQKAWsYASABKAVCBuI/AxIBa1IBayJeCglNdWx0aUNhbGwSUQoDc2VxGAEgAygLMjUuZWR1LnVjaS5pY3MuY - W1iZXIuZW5naW5lLmNvbW1vbi5BY3RvclZpcnR1YWxJZGVudGl0eUII4j8FEgNzZXFSA3NlcSIOCgxFcnJvckNvbW1hbmQiaAoHQ - 29sbGVjdBJdCgd3b3JrZXJzGAEgAygLMjUuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5BY3RvclZpcnR1YWxJZGVud - Gl0eUIM4j8JEgd3b3JrZXJzUgd3b3JrZXJzIhAKDkdlbmVyYXRlTnVtYmVyImAKBUNoYWluElcKBW5leHRzGAEgAygLMjUuZWR1L - nVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5BY3RvclZpcnR1YWxJZGVudGl0eUIK4j8HEgVuZXh0c1IFbmV4dHMiIQoJUmVjd - XJzaW9uEhQKAWkYASABKAVCBuI/AxIBaVIBaSLSAQoWQWRkSW5wdXRDaGFubmVsUmVxdWVzdBJhCgljaGFubmVsSWQYASABKAsyM - C5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkNoYW5uZWxJZGVudGl0eUIR4j8OEgljaGFubmVsSWTwAQFSCWNoYW5uZ - WxJZBJVCgZwb3J0SWQYAiABKAsyLS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLlBvcnRJZGVudGl0eUIO4j8LEgZwb - 3J0SWTwAQFSBnBvcnRJZCLjAQoWQWRkUGFydGl0aW9uaW5nUmVxdWVzdBJMCgN0YWcYASABKAsyLS5lZHUudWNpLmljcy5hbWJlc - i5lbmdpbmUuY29tbW9uLlBoeXNpY2FsTGlua0IL4j8IEgN0YWfwAQFSA3RhZxJ7CgxwYXJ0aXRpb25pbmcYAiABKAsyQS5lZHUud - WNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnNlbmRzZW1hbnRpY3MuUGFydGl0aW9uaW5nQhTiPxESDHBhcnRpdGlvb - mluZ/ABAVIMcGFydGl0aW9uaW5nIswCChFBc3NpZ25Qb3J0UmVxdWVzdBJVCgZwb3J0SWQYASABKAsyLS5lZHUudWNpLmljcy5hb - WJlci5lbmdpbmUuY29tbW9uLlBvcnRJZGVudGl0eUIO4j8LEgZwb3J0SWTwAQFSBnBvcnRJZBIgCgVpbnB1dBgCIAEoCEIK4j8HE - gVpbnB1dFIFaW5wdXQSbQoGc2NoZW1hGAMgAygLMkguZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuQ - XNzaWduUG9ydFJlcXVlc3QuU2NoZW1hRW50cnlCC+I/CBIGc2NoZW1hUgZzY2hlbWEaTwoLU2NoZW1hRW50cnkSGgoDa2V5GAEgA - SgJQgjiPwUSA2tleVIDa2V5EiAKBXZhbHVlGAIgASgJQgriPwcSBXZhbHVlUgV2YWx1ZToCOAEitQEKGUZpbmFsaXplQ2hlY2twb - 2ludFJlcXVlc3QScAoMY2hlY2twb2ludElkGAEgASgLMjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5DaGFubmVsT - WFya2VySWRlbnRpdHlCFOI/ERIMY2hlY2twb2ludElk8AEBUgxjaGVja3BvaW50SWQSJgoHd3JpdGVUbxgCIAEoCUIM4j8JEgd3c - ml0ZVRvUgd3cml0ZVRvIooCChlJbml0aWFsaXplRXhlY3V0b3JSZXF1ZXN0EkEKEHRvdGFsV29ya2VyQ291bnQYASABKAVCFeI/E - hIQdG90YWxXb3JrZXJDb3VudFIQdG90YWxXb3JrZXJDb3VudBJUCg5vcEV4ZWNJbml0SW5mbxgCIAEoCzIULmdvb2dsZS5wcm90b - 2J1Zi5BbnlCFuI/ExIOb3BFeGVjSW5pdEluZm/wAQFSDm9wRXhlY0luaXRJbmZvEikKCGlzU291cmNlGAMgASgIQg3iPwoSCGlzU - 291cmNlUghpc1NvdXJjZRIpCghsYW5ndWFnZRgEIAEoCUIN4j8KEghsYW5ndWFnZVIIbGFuZ3VhZ2UiqQIKFVVwZGF0ZUV4ZWN1d - G9yUmVxdWVzdBJnCgp0YXJnZXRPcElkGAEgASgLMjMuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5QaHlzaWNhbE9wS - WRlbnRpdHlCEuI/DxIKdGFyZ2V0T3BJZPABAVIKdGFyZ2V0T3BJZBJLCgtuZXdFeGVjdXRvchgCIAEoCzIULmdvb2dsZS5wcm90b - 2J1Zi5BbnlCE+I/EBILbmV3RXhlY3V0b3LwAQFSC25ld0V4ZWN1dG9yEloKEXN0YXRlVHJhbnNmZXJGdW5jGAMgASgLMhQuZ29vZ - 2xlLnByb3RvYnVmLkFueUIW4j8TEhFzdGF0ZVRyYW5zZmVyRnVuY1IRc3RhdGVUcmFuc2ZlckZ1bmMiyQEKGFByZXBhcmVDaGVja - 3BvaW50UmVxdWVzdBJwCgxjaGVja3BvaW50SWQYASABKAsyNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkNoYW5uZ - WxNYXJrZXJJZGVudGl0eUIU4j8REgxjaGVja3BvaW50SWTwAQFSDGNoZWNrcG9pbnRJZBI7Cg5lc3RpbWF0aW9uT25seRgCIAEoC - EIT4j8QEg5lc3RpbWF0aW9uT25seVIOZXN0aW1hdGlvbk9ubHkijwEKFlF1ZXJ5U3RhdGlzdGljc1JlcXVlc3QSdQoPZmlsdGVyQ - nlXb3JrZXJzGAEgAygLMjUuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5BY3RvclZpcnR1YWxJZGVudGl0eUIU4j8RE - g9maWx0ZXJCeVdvcmtlcnNSD2ZpbHRlckJ5V29ya2VycypnChFDaGFubmVsTWFya2VyVHlwZRItChFSRVFVSVJFX0FMSUdOTUVOV - BAAGhbiPxMSEVJFUVVJUkVfQUxJR05NRU5UEiMKDE5PX0FMSUdOTUVOVBABGhHiPw4SDE5PX0FMSUdOTUVOVCp6ChJDb25zb2xlT - WVzc2FnZVR5cGUSFQoFUFJJTlQQABoK4j8HEgVQUklOVBIVCgVFUlJPUhABGgriPwcSBUVSUk9SEhkKB0NPTU1BTkQQAhoM4j8JE - gdDT01NQU5EEhsKCERFQlVHR0VSEAMaDeI/ChIIREVCVUdHRVJCCeI/BkgAWAB4AWIGcHJvdG8z""" - ).mkString) - lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { - val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) - _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) - } - lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { - val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) - com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( - edu.uci.ics.amber.engine.common.virtualidentity.VirtualidentityProto.javaDescriptor, - edu.uci.ics.amber.engine.common.workflow.WorkflowProto.javaDescriptor, - edu.uci.ics.amber.engine.architecture.worker.statistics.StatisticsProto.javaDescriptor, - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningsProto.javaDescriptor, - scalapb.options.ScalapbProto.javaDescriptor, - com.google.protobuf.timestamp.TimestampProto.javaDescriptor, - com.google.protobuf.any.AnyProto.javaDescriptor - )) - } - @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") - def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor -} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controllerservice/ControllerServiceFs2Grpc.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controllerservice/ControllerServiceFs2Grpc.scala deleted file mode 100644 index 2def344eb0a..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controllerservice/ControllerServiceFs2Grpc.scala +++ /dev/null @@ -1,124 +0,0 @@ -package edu.uci.ics.amber.engine.architecture.rpc.controllerservice - -import _root_.cats.syntax.all._ - -trait ControllerServiceFs2Grpc[F[_], A] { - def retrieveWorkflowState(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse] - def propagateChannelMarker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse] - def takeGlobalCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse] - def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse] - def consoleMessageTriggered(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def portCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def startWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse] - def resumeWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def pauseWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def workerStateUpdated(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def workerExecutionCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def linkWorkers(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def controllerInitiateQueryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def retryWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] -} - -object ControllerServiceFs2Grpc extends _root_.fs2.grpc.GeneratedCompanion[ControllerServiceFs2Grpc] { - - def mkClient[F[_]: _root_.cats.effect.Async, A](dispatcher: _root_.cats.effect.std.Dispatcher[F], channel: _root_.io.grpc.Channel, mkMetadata: A => F[_root_.io.grpc.Metadata], clientOptions: _root_.fs2.grpc.client.ClientOptions): ControllerServiceFs2Grpc[F, A] = new ControllerServiceFs2Grpc[F, A] { - def retrieveWorkflowState(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_RETRIEVE_WORKFLOW_STATE, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def propagateChannelMarker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_PROPAGATE_CHANNEL_MARKER, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def takeGlobalCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_TAKE_GLOBAL_CHECKPOINT, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_DEBUG_COMMAND, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_EVALUATE_PYTHON_EXPRESSION, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def consoleMessageTriggered(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_CONSOLE_MESSAGE_TRIGGERED, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def portCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_PORT_COMPLETED, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def startWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_START_WORKFLOW, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def resumeWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_RESUME_WORKFLOW, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def pauseWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_PAUSE_WORKFLOW, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def workerStateUpdated(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_WORKER_STATE_UPDATED, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def workerExecutionCompleted(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_WORKER_EXECUTION_COMPLETED, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def linkWorkers(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_LINK_WORKERS, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def controllerInitiateQueryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_CONTROLLER_INITIATE_QUERY_STATISTICS, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def retryWorkflow(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_RETRY_WORKFLOW, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - } - - protected def serviceBinding[F[_]: _root_.cats.effect.Async, A](dispatcher: _root_.cats.effect.std.Dispatcher[F], serviceImpl: ControllerServiceFs2Grpc[F, A], mkCtx: _root_.io.grpc.Metadata => F[A], serverOptions: _root_.fs2.grpc.server.ServerOptions): _root_.io.grpc.ServerServiceDefinition = { - _root_.io.grpc.ServerServiceDefinition - .builder(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.SERVICE) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_RETRIEVE_WORKFLOW_STATE, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.retrieveWorkflowState(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_PROPAGATE_CHANNEL_MARKER, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PropagateChannelMarkerRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.propagateChannelMarker(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_TAKE_GLOBAL_CHECKPOINT, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.TakeGlobalCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.takeGlobalCheckpoint(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_DEBUG_COMMAND, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.debugCommand(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_EVALUATE_PYTHON_EXPRESSION, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.evaluatePythonExpression(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_CONSOLE_MESSAGE_TRIGGERED, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessageTriggeredRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.consoleMessageTriggered(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_PORT_COMPLETED, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PortCompletedRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.portCompleted(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_START_WORKFLOW, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.startWorkflow(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_RESUME_WORKFLOW, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.resumeWorkflow(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_PAUSE_WORKFLOW, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.pauseWorkflow(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_WORKER_STATE_UPDATED, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.WorkerStateUpdatedRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.workerStateUpdated(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_WORKER_EXECUTION_COMPLETED, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.workerExecutionCompleted(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_LINK_WORKERS, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.LinkWorkersRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.linkWorkers(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_CONTROLLER_INITIATE_QUERY_STATISTICS, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.QueryStatisticsRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.controllerInitiateQueryStatistics(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_RETRY_WORKFLOW, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.RetryWorkflowRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.retryWorkflow(r, _)))) - .build() - } - -} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controllerservice/ControllerserviceProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controllerservice/ControllerserviceProto.scala deleted file mode 100644 index 92a2738a8f4..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controllerservice/ControllerserviceProto.scala +++ /dev/null @@ -1,67 +0,0 @@ -// Generated by the Scala Plugin for the Protocol Buffer Compiler. -// Do not edit! -// -// Protofile syntax: PROTO3 - -package edu.uci.ics.amber.engine.architecture.rpc.controllerservice - -object ControllerserviceProto extends _root_.scalapb.GeneratedFileObject { - lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlcommandsProto, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlreturnsProto, - scalapb.options.ScalapbProto - ) - lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty - private lazy val ProtoBytes: _root_.scala.Array[Byte] = - scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( - """CkFlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy9jb250cm9sbGVyc2VydmljZS5wcm90bxIpZWR1L - nVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMaP2VkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9hcmNoaXRlY3R1c - mUvcnBjL2NvbnRyb2xjb21tYW5kcy5wcm90bxo+ZWR1L3VjaS9pY3MvYW1iZXIvZW5naW5lL2FyY2hpdGVjdHVyZS9ycGMvY29ud - HJvbHJldHVybnMucHJvdG8aFXNjYWxhcGIvc2NhbGFwYi5wcm90bzLdEQoRQ29udHJvbGxlclNlcnZpY2USmgEKFVJldHJpZXZlV - 29ya2Zsb3dTdGF0ZRI3LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmVxdWVzdBpILmVkd - S51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlJldHJpZXZlV29ya2Zsb3dTdGF0ZVJlc3BvbnNlEq0BChZQc - m9wYWdhdGVDaGFubmVsTWFya2VyEkguZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuUHJvcGFnYXRlQ - 2hhbm5lbE1hcmtlclJlcXVlc3QaSS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Qcm9wYWdhdGVDa - GFubmVsTWFya2VyUmVzcG9uc2USpwEKFFRha2VHbG9iYWxDaGVja3BvaW50EkYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY - 2hpdGVjdHVyZS5ycGMuVGFrZUdsb2JhbENoZWNrcG9pbnRSZXF1ZXN0GkcuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpd - GVjdHVyZS5ycGMuVGFrZUdsb2JhbENoZWNrcG9pbnRSZXNwb25zZRKGAQoMRGVidWdDb21tYW5kEj4uZWR1LnVjaS5pY3MuYW1iZ - XIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRGVidWdDb21tYW5kUmVxdWVzdBo2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hc - mNoaXRlY3R1cmUucnBjLkVtcHR5UmV0dXJuErMBChhFdmFsdWF0ZVB5dGhvbkV4cHJlc3Npb24SSi5lZHUudWNpLmljcy5hbWJlc - i5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FdmFsdWF0ZVB5dGhvbkV4cHJlc3Npb25SZXF1ZXN0GksuZWR1LnVjaS5pY3MuYW1iZ - XIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRXZhbHVhdGVQeXRob25FeHByZXNzaW9uUmVzcG9uc2USnAEKF0NvbnNvbGVNZXNzY - WdlVHJpZ2dlcmVkEkkuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuQ29uc29sZU1lc3NhZ2VUcmlnZ - 2VyZWRSZXF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm4SiAEKDVBvc - nRDb21wbGV0ZWQSPy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Qb3J0Q29tcGxldGVkUmVxdWVzd - Bo2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmV0dXJuEooBCg1TdGFydFdvcmtmbG93E - jcuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXF1ZXN0GkAuZWR1LnVjaS5pY3MuYW1iZ - XIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuU3RhcnRXb3JrZmxvd1Jlc3BvbnNlEoEBCg5SZXN1bWVXb3JrZmxvdxI3LmVkdS51Y - 2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmVxdWVzdBo2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZ - S5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmV0dXJuEoABCg1QYXVzZVdvcmtmbG93EjcuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lL - mFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuR - W1wdHlSZXR1cm4SkgEKEldvcmtlclN0YXRlVXBkYXRlZBJELmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUuc - nBjLldvcmtlclN0YXRlVXBkYXRlZFJlcXVlc3QaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Fb - XB0eVJldHVybhKLAQoYV29ya2VyRXhlY3V0aW9uQ29tcGxldGVkEjcuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjd - HVyZS5ycGMuRW1wdHlSZXF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1c - m4ShAEKC0xpbmtXb3JrZXJzEj0uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuTGlua1dvcmtlcnNSZ - XF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm4SngEKIUNvbnRyb2xsZ - XJJbml0aWF0ZVF1ZXJ5U3RhdGlzdGljcxJBLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlF1ZXJ5U - 3RhdGlzdGljc1JlcXVlc3QaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJldHVybhKIA - QoNUmV0cnlXb3JrZmxvdxI/LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlJldHJ5V29ya2Zsb3dSZ - XF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm5CCeI/BkgAWAB4AWIGc - HJvdG8z""" - ).mkString) - lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { - val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) - _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) - } - lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { - val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) - com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlcommandsProto.javaDescriptor, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlreturnsProto.javaDescriptor, - scalapb.options.ScalapbProto.javaDescriptor - )) - } - @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") - def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor -} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ControlreturnsProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ControlreturnsProto.scala deleted file mode 100644 index 93b48d4f43a..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/ControlreturnsProto.scala +++ /dev/null @@ -1,112 +0,0 @@ -// Generated by the Scala Plugin for the Protocol Buffer Compiler. -// Do not edit! -// -// Protofile syntax: PROTO3 - -package edu.uci.ics.amber.engine.architecture.rpc.controlreturns - -object ControlreturnsProto extends _root_.scalapb.GeneratedFileObject { - lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( - edu.uci.ics.amber.engine.architecture.worker.statistics.StatisticsProto, - scalapb.options.ScalapbProto - ) - lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = - Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlReturnMessage, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlError, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.RetrieveWorkflowStateResponse, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.PropagateChannelMarkerResponse, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TakeGlobalCheckpointResponse, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TypedValue, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatePythonExpressionResponse, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StartWorkflowResponse, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse - ) - private lazy val ProtoBytes: _root_.scala.Array[Byte] = - scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( - """Cj5lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy9jb250cm9scmV0dXJucy5wcm90bxIpZWR1LnVja - S5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMaPWVkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9hcmNoaXRlY3R1cmUvd - 29ya2VyL3N0YXRpc3RpY3MucHJvdG8aFXNjYWxhcGIvc2NhbGFwYi5wcm90byKiDgoNQ29udHJvbFJldHVybhK0AQodcmV0cmlld - mVXb3JrZmxvd1N0YXRlUmVzcG9uc2UYASABKAsySC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5SZ - XRyaWV2ZVdvcmtmbG93U3RhdGVSZXNwb25zZUIi4j8fEh1yZXRyaWV2ZVdvcmtmbG93U3RhdGVSZXNwb25zZUgAUh1yZXRyaWV2Z - VdvcmtmbG93U3RhdGVSZXNwb25zZRK4AQoecHJvcGFnYXRlQ2hhbm5lbE1hcmtlclJlc3BvbnNlGAIgASgLMkkuZWR1LnVjaS5pY - 3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuUHJvcGFnYXRlQ2hhbm5lbE1hcmtlclJlc3BvbnNlQiPiPyASHnByb3BhZ - 2F0ZUNoYW5uZWxNYXJrZXJSZXNwb25zZUgAUh5wcm9wYWdhdGVDaGFubmVsTWFya2VyUmVzcG9uc2USsAEKHHRha2VHbG9iYWxDa - GVja3BvaW50UmVzcG9uc2UYAyABKAsyRy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5UYWtlR2xvY - mFsQ2hlY2twb2ludFJlc3BvbnNlQiHiPx4SHHRha2VHbG9iYWxDaGVja3BvaW50UmVzcG9uc2VIAFIcdGFrZUdsb2JhbENoZWNrc - G9pbnRSZXNwb25zZRLAAQogZXZhbHVhdGVQeXRob25FeHByZXNzaW9uUmVzcG9uc2UYBCABKAsySy5lZHUudWNpLmljcy5hbWJlc - i5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FdmFsdWF0ZVB5dGhvbkV4cHJlc3Npb25SZXNwb25zZUIl4j8iEiBldmFsdWF0ZVB5d - GhvbkV4cHJlc3Npb25SZXNwb25zZUgAUiBldmFsdWF0ZVB5dGhvbkV4cHJlc3Npb25SZXNwb25zZRKUAQoVc3RhcnRXb3JrZmxvd - 1Jlc3BvbnNlGAUgASgLMkAuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuU3RhcnRXb3JrZmxvd1Jlc - 3BvbnNlQhriPxcSFXN0YXJ0V29ya2Zsb3dSZXNwb25zZUgAUhVzdGFydFdvcmtmbG93UmVzcG9uc2USjAEKE3dvcmtlclN0YXRlU - mVzcG9uc2UYMiABKAsyPi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Xb3JrZXJTdGF0ZVJlc3Bvb - nNlQhjiPxUSE3dvcmtlclN0YXRlUmVzcG9uc2VIAFITd29ya2VyU3RhdGVSZXNwb25zZRKUAQoVd29ya2VyTWV0cmljc1Jlc3Bvb - nNlGDMgASgLMkAuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuV29ya2VyTWV0cmljc1Jlc3BvbnNlQ - hriPxcSFXdvcmtlck1ldHJpY3NSZXNwb25zZUgAUhV3b3JrZXJNZXRyaWNzUmVzcG9uc2USqAEKGmZpbmFsaXplQ2hlY2twb2lud - FJlc3BvbnNlGDQgASgLMkUuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRmluYWxpemVDaGVja3Bva - W50UmVzcG9uc2VCH+I/HBIaZmluYWxpemVDaGVja3BvaW50UmVzcG9uc2VIAFIaZmluYWxpemVDaGVja3BvaW50UmVzcG9uc2USW - woFZXJyb3IYZSABKAsyNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Db250cm9sRXJyb3JCCuI/B - xIFZXJyb3JIAFIFZXJyb3ISbAoLZW1wdHlSZXR1cm4YZiABKAsyNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0d - XJlLnJwYy5FbXB0eVJldHVybkIQ4j8NEgtlbXB0eVJldHVybkgAUgtlbXB0eVJldHVybhJ4Cg5zdHJpbmdSZXNwb25zZRhnIAEoC - zI5LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlN0cmluZ1Jlc3BvbnNlQhPiPxASDnN0cmluZ1Jlc - 3BvbnNlSABSDnN0cmluZ1Jlc3BvbnNlEmwKC2ludFJlc3BvbnNlGGggASgLMjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY - 2hpdGVjdHVyZS5ycGMuSW50UmVzcG9uc2VCEOI/DRILaW50UmVzcG9uc2VIAFILaW50UmVzcG9uc2VCDgoMc2VhbGVkX3ZhbHVlI - g0KC0VtcHR5UmV0dXJuIpICCgxDb250cm9sRXJyb3ISNQoMZXJyb3JNZXNzYWdlGAEgASgJQhHiPw4SDGVycm9yTWVzc2FnZVIMZ - XJyb3JNZXNzYWdlEjUKDGVycm9yRGV0YWlscxgCIAEoCUIR4j8OEgxlcnJvckRldGFpbHNSDGVycm9yRGV0YWlscxIvCgpzdGFja - 1RyYWNlGAMgASgJQg/iPwwSCnN0YWNrVHJhY2VSCnN0YWNrVHJhY2USYwoIbGFuZ3VhZ2UYBCABKA4yOC5lZHUudWNpLmljcy5hb - WJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FcnJvckxhbmd1YWdlQg3iPwoSCGxhbmd1YWdlUghsYW5ndWFnZSLzAQoQUmV0d - XJuSW52b2NhdGlvbhIsCgljb21tYW5kSWQYASABKANCDuI/CxIJY29tbWFuZElkUgljb21tYW5kSWQSbwoLcmV0dXJuVmFsdWUYA - iABKAsyOC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Db250cm9sUmV0dXJuQhPiPxASC3JldHVyb - lZhbHVl8AEBUgtyZXR1cm5WYWx1ZTpA4j89CjtlZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLmFtYmVybWVzc2FnZS5Db - 250cm9sUGF5bG9hZCIyCg5TdHJpbmdSZXNwb25zZRIgCgV2YWx1ZRgBIAEoCUIK4j8HEgV2YWx1ZVIFdmFsdWUiLwoLSW50UmVzc - G9uc2USIAoFdmFsdWUYASABKAVCCuI/BxIFdmFsdWVSBXZhbHVlIuYBCh1SZXRyaWV2ZVdvcmtmbG93U3RhdGVSZXNwb25zZRJ1C - gVzdGF0ZRgBIAMoCzJTLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlJldHJpZXZlV29ya2Zsb3dTd - GF0ZVJlc3BvbnNlLlN0YXRlRW50cnlCCuI/BxIFc3RhdGVSBXN0YXRlGk4KClN0YXRlRW50cnkSGgoDa2V5GAEgASgJQgjiPwUSA - 2tleVIDa2V5EiAKBXZhbHVlGAIgASgJQgriPwcSBXZhbHVlUgV2YWx1ZToCOAEiOwoaRmluYWxpemVDaGVja3BvaW50UmVzcG9uc - 2USHQoEc2l6ZRgBIAEoA0IJ4j8GEgRzaXplUgRzaXplIq0CCh5Qcm9wYWdhdGVDaGFubmVsTWFya2VyUmVzcG9uc2USfgoHcmV0d - XJucxgBIAMoCzJWLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlByb3BhZ2F0ZUNoYW5uZWxNYXJrZ - XJSZXNwb25zZS5SZXR1cm5zRW50cnlCDOI/CRIHcmV0dXJuc1IHcmV0dXJucxqKAQoMUmV0dXJuc0VudHJ5EhoKA2tleRgBIAEoC - UII4j8FEgNrZXlSA2tleRJaCgV2YWx1ZRgCIAEoCzI4LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjL - kNvbnRyb2xSZXR1cm5CCuI/BxIFdmFsdWVSBXZhbHVlOgI4ASJMChxUYWtlR2xvYmFsQ2hlY2twb2ludFJlc3BvbnNlEiwKCXRvd - GFsU2l6ZRgBIAEoA0IO4j8LEgl0b3RhbFNpemVSCXRvdGFsU2l6ZSL1AQoKVHlwZWRWYWx1ZRIvCgpleHByZXNzaW9uGAEgASgJQ - g/iPwwSCmV4cHJlc3Npb25SCmV4cHJlc3Npb24SKgoJdmFsdWVfcmVmGAIgASgJQg3iPwoSCHZhbHVlUmVmUgh2YWx1ZVJlZhIqC - gl2YWx1ZV9zdHIYAyABKAlCDeI/ChIIdmFsdWVTdHJSCHZhbHVlU3RyEi0KCnZhbHVlX3R5cGUYBCABKAlCDuI/CxIJdmFsdWVUe - XBlUgl2YWx1ZVR5cGUSLwoKZXhwYW5kYWJsZRgFIAEoCEIP4j8MEgpleHBhbmRhYmxlUgpleHBhbmRhYmxlItEBCg5FdmFsdWF0Z - WRWYWx1ZRJXCgV2YWx1ZRgBIAEoCzI1LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlR5cGVkVmFsd - WVCCuI/BxIFdmFsdWVSBXZhbHVlEmYKCmF0dHJpYnV0ZXMYAiADKAsyNS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0Z - WN0dXJlLnJwYy5UeXBlZFZhbHVlQg/iPwwSCmF0dHJpYnV0ZXNSCmF0dHJpYnV0ZXMiggEKIEV2YWx1YXRlUHl0aG9uRXhwcmVzc - 2lvblJlc3BvbnNlEl4KBnZhbHVlcxgBIAMoCzI5LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkV2Y - Wx1YXRlZFZhbHVlQgviPwgSBnZhbHVlc1IGdmFsdWVzIpgBChVTdGFydFdvcmtmbG93UmVzcG9uc2USfwoNd29ya2Zsb3dTdGF0Z - RgBIAEoDjJCLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLldvcmtmbG93QWdncmVnYXRlZFN0YXRlQ - hXiPxISDXdvcmtmbG93U3RhdGXwAQFSDXdvcmtmbG93U3RhdGUidQoTV29ya2VyU3RhdGVSZXNwb25zZRJeCgVzdGF0ZRgBIAEoD - jI5LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUud29ya2VyLldvcmtlclN0YXRlQg3iPwoSBXN0YXRl8AEBU - gVzdGF0ZSJ/ChVXb3JrZXJNZXRyaWNzUmVzcG9uc2USZgoHbWV0cmljcxgBIAEoCzI7LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZ - S5hcmNoaXRlY3R1cmUud29ya2VyLldvcmtlck1ldHJpY3NCD+I/DBIHbWV0cmljc/ABAVIHbWV0cmljcyo/Cg1FcnJvckxhbmd1Y - WdlEhcKBlBZVEhPThAAGgviPwgSBlBZVEhPThIVCgVTQ0FMQRABGgriPwcSBVNDQUxBKq8CChdXb3JrZmxvd0FnZ3JlZ2F0ZWRTd - GF0ZRIlCg1VTklOSVRJQUxJWkVEEAAaEuI/DxINVU5JTklUSUFMSVpFRBIVCgVSRUFEWRABGgriPwcSBVJFQURZEhkKB1JVTk5JT - kcQAhoM4j8JEgdSVU5OSU5HEhkKB1BBVVNJTkcQAxoM4j8JEgdQQVVTSU5HEhcKBlBBVVNFRBAEGgviPwgSBlBBVVNFRBIbCghSR - VNVTUlORxAFGg3iPwoSCFJFU1VNSU5HEh0KCUNPTVBMRVRFRBAGGg7iPwsSCUNPTVBMRVRFRBIXCgZGQUlMRUQQBxoL4j8IEgZGQ - UlMRUQSGQoHVU5LTk9XThAIGgziPwkSB1VOS05PV04SFwoGS0lMTEVEEAkaC+I/CBIGS0lMTEVEQgniPwZIAFgAeAFiBnByb3RvM - w==""" - ).mkString) - lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { - val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) - _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) - } - lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { - val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) - com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( - edu.uci.ics.amber.engine.architecture.worker.statistics.StatisticsProto.javaDescriptor, - scalapb.options.ScalapbProto.javaDescriptor - )) - } - @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") - def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor -} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/testerservice/RPCTesterFs2Grpc.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/testerservice/RPCTesterFs2Grpc.scala deleted file mode 100644 index 41885b80979..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/testerservice/RPCTesterFs2Grpc.scala +++ /dev/null @@ -1,89 +0,0 @@ -package edu.uci.ics.amber.engine.architecture.rpc.testerservice - -import _root_.cats.syntax.all._ - -trait RPCTesterFs2Grpc[F[_], A] { - def sendPing(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] - def sendPong(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] - def sendNested(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] - def sendPass(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] - def sendErrorCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] - def sendRecursion(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] - def sendCollect(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] - def sendGenerateNumber(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] - def sendMultiCall(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] - def sendChain(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] -} - -object RPCTesterFs2Grpc extends _root_.fs2.grpc.GeneratedCompanion[RPCTesterFs2Grpc] { - - def mkClient[F[_]: _root_.cats.effect.Async, A](dispatcher: _root_.cats.effect.std.Dispatcher[F], channel: _root_.io.grpc.Channel, mkMetadata: A => F[_root_.io.grpc.Metadata], clientOptions: _root_.fs2.grpc.client.ClientOptions): RPCTesterFs2Grpc[F, A] = new RPCTesterFs2Grpc[F, A] { - def sendPing(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_PING, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def sendPong(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_PONG, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def sendNested(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_NESTED, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def sendPass(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_PASS, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def sendErrorCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_ERROR_COMMAND, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def sendRecursion(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_RECURSION, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def sendCollect(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_COLLECT, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def sendGenerateNumber(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_GENERATE_NUMBER, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def sendMultiCall(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_MULTI_CALL, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def sendChain(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_CHAIN, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - } - - protected def serviceBinding[F[_]: _root_.cats.effect.Async, A](dispatcher: _root_.cats.effect.std.Dispatcher[F], serviceImpl: RPCTesterFs2Grpc[F, A], mkCtx: _root_.io.grpc.Metadata => F[A], serverOptions: _root_.fs2.grpc.server.ServerOptions): _root_.io.grpc.ServerServiceDefinition = { - _root_.io.grpc.ServerServiceDefinition - .builder(edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.SERVICE) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_PING, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Ping, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendPing(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_PONG, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pong, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendPong(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_NESTED, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Nested, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendNested(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_PASS, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Pass, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendPass(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_ERROR_COMMAND, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ErrorCommand, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendErrorCommand(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_RECURSION, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Recursion, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendRecursion(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_COLLECT, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Collect, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendCollect(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_GENERATE_NUMBER, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.GenerateNumber, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.IntResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendGenerateNumber(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_MULTI_CALL, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.MultiCall, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendMultiCall(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.METHOD_SEND_CHAIN, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.Chain, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.StringResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.sendChain(r, _)))) - .build() - } - -} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/testerservice/TesterserviceProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/testerservice/TesterserviceProto.scala deleted file mode 100644 index 59c9506043d..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/testerservice/TesterserviceProto.scala +++ /dev/null @@ -1,54 +0,0 @@ -// Generated by the Scala Plugin for the Protocol Buffer Compiler. -// Do not edit! -// -// Protofile syntax: PROTO3 - -package edu.uci.ics.amber.engine.architecture.rpc.testerservice - -object TesterserviceProto extends _root_.scalapb.GeneratedFileObject { - lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlcommandsProto, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlreturnsProto, - scalapb.options.ScalapbProto - ) - lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty - private lazy val ProtoBytes: _root_.scala.Array[Byte] = - scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( - """Cj1lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy90ZXN0ZXJzZXJ2aWNlLnByb3RvEillZHUudWNpL - mljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYxo/ZWR1L3VjaS9pY3MvYW1iZXIvZW5naW5lL2FyY2hpdGVjdHVyZS9yc - GMvY29udHJvbGNvbW1hbmRzLnByb3RvGj5lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy9jb250cm9sc - mV0dXJucy5wcm90bxoVc2NhbGFwYi9zY2FsYXBiLnByb3RvMo4KCglSUENUZXN0ZXISdQoIU2VuZFBpbmcSLy5lZHUudWNpLmljc - y5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5QaW5nGjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZ - S5ycGMuSW50UmVzcG9uc2UiABJ1CghTZW5kUG9uZxIvLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjL - lBvbmcaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5JbnRSZXNwb25zZSIAEnwKClNlbmROZXN0Z - WQSMS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5OZXN0ZWQaOS5lZHUudWNpLmljcy5hbWJlci5lb - mdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5TdHJpbmdSZXNwb25zZSIAEngKCFNlbmRQYXNzEi8uZWR1LnVjaS5pY3MuYW1iZXIuZW5na - W5lLmFyY2hpdGVjdHVyZS5ycGMuUGFzcxo5LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlN0cmluZ - 1Jlc3BvbnNlIgASiAEKEFNlbmRFcnJvckNvbW1hbmQSNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwY - y5FcnJvckNvbW1hbmQaOS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5TdHJpbmdSZXNwb25zZSIAE - oIBCg1TZW5kUmVjdXJzaW9uEjQuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuUmVjdXJzaW9uGjkuZ - WR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuU3RyaW5nUmVzcG9uc2UiABJ+CgtTZW5kQ29sbGVjdBIyL - mVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkNvbGxlY3QaOS5lZHUudWNpLmljcy5hbWJlci5lbmdpb - mUuYXJjaGl0ZWN0dXJlLnJwYy5TdHJpbmdSZXNwb25zZSIAEokBChJTZW5kR2VuZXJhdGVOdW1iZXISOS5lZHUudWNpLmljcy5hb - WJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5HZW5lcmF0ZU51bWJlcho2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoa - XRlY3R1cmUucnBjLkludFJlc3BvbnNlIgASggEKDVNlbmRNdWx0aUNhbGwSNC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJja - Gl0ZWN0dXJlLnJwYy5NdWx0aUNhbGwaOS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5TdHJpbmdSZ - XNwb25zZSIAEnoKCVNlbmRDaGFpbhIwLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkNoYWluGjkuZ - WR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuU3RyaW5nUmVzcG9uc2UiAEIJ4j8GSABYAHgBYgZwcm90b - zM=""" - ).mkString) - lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { - val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) - _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) - } - lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { - val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) - com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlcommandsProto.javaDescriptor, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlreturnsProto.javaDescriptor, - scalapb.options.ScalapbProto.javaDescriptor - )) - } - @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") - def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor -} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/workerservice/WorkerServiceFs2Grpc.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/workerservice/WorkerServiceFs2Grpc.scala deleted file mode 100644 index f14ca1648e2..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/workerservice/WorkerServiceFs2Grpc.scala +++ /dev/null @@ -1,138 +0,0 @@ -package edu.uci.ics.amber.engine.architecture.rpc.workerservice - -import _root_.cats.syntax.all._ - -trait WorkerServiceFs2Grpc[F[_], A] { - def addInputChannel(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def addPartitioning(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def assignPort(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def finalizeCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse] - def flushNetworkBuffer(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def initializeExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def openExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def pauseWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] - def prepareCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def queryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse] - def resumeWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] - def retrieveState(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def retryCurrentTuple(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def startWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] - def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] - def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] - def noOperation(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] -} - -object WorkerServiceFs2Grpc extends _root_.fs2.grpc.GeneratedCompanion[WorkerServiceFs2Grpc] { - - def mkClient[F[_]: _root_.cats.effect.Async, A](dispatcher: _root_.cats.effect.std.Dispatcher[F], channel: _root_.io.grpc.Channel, mkMetadata: A => F[_root_.io.grpc.Metadata], clientOptions: _root_.fs2.grpc.client.ClientOptions): WorkerServiceFs2Grpc[F, A] = new WorkerServiceFs2Grpc[F, A] { - def addInputChannel(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_ADD_INPUT_CHANNEL, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def addPartitioning(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_ADD_PARTITIONING, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def assignPort(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_ASSIGN_PORT, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def finalizeCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_FINALIZE_CHECKPOINT, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def flushNetworkBuffer(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_FLUSH_NETWORK_BUFFER, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def initializeExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_INITIALIZE_EXECUTOR, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def openExecutor(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_OPEN_EXECUTOR, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def pauseWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_PAUSE_WORKER, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def prepareCheckpoint(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_PREPARE_CHECKPOINT, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def queryStatistics(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_QUERY_STATISTICS, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def resumeWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_RESUME_WORKER, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def retrieveState(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_RETRIEVE_STATE, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def retryCurrentTuple(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_RETRY_CURRENT_TUPLE, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def startWorker(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_START_WORKER, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def debugCommand(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_DEBUG_COMMAND, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def evaluatePythonExpression(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_EVALUATE_PYTHON_EXPRESSION, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - def noOperation(request: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, ctx: A): F[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn] = { - mkMetadata(ctx).flatMap { m => - _root_.fs2.grpc.client.Fs2ClientCall[F](channel, edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_NO_OPERATION, dispatcher, clientOptions).flatMap(_.unaryToUnaryCall(request, m)) - } - } - } - - protected def serviceBinding[F[_]: _root_.cats.effect.Async, A](dispatcher: _root_.cats.effect.std.Dispatcher[F], serviceImpl: WorkerServiceFs2Grpc[F, A], mkCtx: _root_.io.grpc.Metadata => F[A], serverOptions: _root_.fs2.grpc.server.ServerOptions): _root_.io.grpc.ServerServiceDefinition = { - _root_.io.grpc.ServerServiceDefinition - .builder(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.SERVICE) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_ADD_INPUT_CHANNEL, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddInputChannelRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.addInputChannel(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_ADD_PARTITIONING, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AddPartitioningRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.addPartitioning(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_ASSIGN_PORT, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AssignPortRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.assignPort(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_FINALIZE_CHECKPOINT, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.FinalizeCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.FinalizeCheckpointResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.finalizeCheckpoint(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_FLUSH_NETWORK_BUFFER, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.flushNetworkBuffer(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_INITIALIZE_EXECUTOR, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.InitializeExecutorRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.initializeExecutor(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_OPEN_EXECUTOR, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.openExecutor(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_PAUSE_WORKER, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.pauseWorker(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_PREPARE_CHECKPOINT, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.PrepareCheckpointRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.prepareCheckpoint(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_QUERY_STATISTICS, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerMetricsResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.queryStatistics(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_RESUME_WORKER, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.resumeWorker(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_RETRIEVE_STATE, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.retrieveState(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_RETRY_CURRENT_TUPLE, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.retryCurrentTuple(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_START_WORKER, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkerStateResponse]((r, m) => mkCtx(m).flatMap(serviceImpl.startWorker(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_DEBUG_COMMAND, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.DebugCommandRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.debugCommand(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_EVALUATE_PYTHON_EXPRESSION, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EvaluatePythonExpressionRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue]((r, m) => mkCtx(m).flatMap(serviceImpl.evaluatePythonExpression(r, _)))) - .addMethod(edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_NO_OPERATION, _root_.fs2.grpc.server.Fs2ServerCallHandler[F](dispatcher, serverOptions).unaryToUnaryCall[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn]((r, m) => mkCtx(m).flatMap(serviceImpl.noOperation(r, _)))) - .build() - } - -} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/workerservice/WorkerserviceProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/workerservice/WorkerserviceProto.scala deleted file mode 100644 index 0151921240e..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/workerservice/WorkerserviceProto.scala +++ /dev/null @@ -1,68 +0,0 @@ -// Generated by the Scala Plugin for the Protocol Buffer Compiler. -// Do not edit! -// -// Protofile syntax: PROTO3 - -package edu.uci.ics.amber.engine.architecture.rpc.workerservice - -object WorkerserviceProto extends _root_.scalapb.GeneratedFileObject { - lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlcommandsProto, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlreturnsProto, - scalapb.options.ScalapbProto - ) - lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty - private lazy val ProtoBytes: _root_.scala.Array[Byte] = - scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( - """Cj1lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy93b3JrZXJzZXJ2aWNlLnByb3RvEillZHUudWNpL - mljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYxo/ZWR1L3VjaS9pY3MvYW1iZXIvZW5naW5lL2FyY2hpdGVjdHVyZS9yc - GMvY29udHJvbGNvbW1hbmRzLnByb3RvGj5lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy9jb250cm9sc - mV0dXJucy5wcm90bxoVc2NhbGFwYi9zY2FsYXBiLnByb3RvMukSCg1Xb3JrZXJTZXJ2aWNlEowBCg9BZGRJbnB1dENoYW5uZWwSQ - S5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5BZGRJbnB1dENoYW5uZWxSZXF1ZXN0GjYuZWR1LnVja - S5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm4SjAEKD0FkZFBhcnRpdGlvbmluZxJBLmVkdS51Y - 2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkFkZFBhcnRpdGlvbmluZ1JlcXVlc3QaNi5lZHUudWNpLmljcy5hb - WJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJldHVybhKCAQoKQXNzaWduUG9ydBI8LmVkdS51Y2kuaWNzLmFtYmVyL - mVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkFzc2lnblBvcnRSZXF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpd - GVjdHVyZS5ycGMuRW1wdHlSZXR1cm4SoQEKEkZpbmFsaXplQ2hlY2twb2ludBJELmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hc - mNoaXRlY3R1cmUucnBjLkZpbmFsaXplQ2hlY2twb2ludFJlcXVlc3QaRS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0Z - WN0dXJlLnJwYy5GaW5hbGl6ZUNoZWNrcG9pbnRSZXNwb25zZRKFAQoSRmx1c2hOZXR3b3JrQnVmZmVyEjcuZWR1LnVjaS5pY3MuY - W1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXF1ZXN0GjYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpd - GVjdHVyZS5ycGMuRW1wdHlSZXR1cm4SkgEKEkluaXRpYWxpemVFeGVjdXRvchJELmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hc - mNoaXRlY3R1cmUucnBjLkluaXRpYWxpemVFeGVjdXRvclJlcXVlc3QaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0Z - WN0dXJlLnJwYy5FbXB0eVJldHVybhJ/CgxPcGVuRXhlY3V0b3ISNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0d - XJlLnJwYy5FbXB0eVJlcXVlc3QaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJldHVyb - hKGAQoLUGF1c2VXb3JrZXISNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJlcXVlc3QaP - i5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Xb3JrZXJTdGF0ZVJlc3BvbnNlEpABChFQcmVwYXJlQ - 2hlY2twb2ludBJDLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLlByZXBhcmVDaGVja3BvaW50UmVxd - WVzdBo2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmV0dXJuEowBCg9RdWVyeVN0YXRpc - 3RpY3MSNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJlcXVlc3QaQC5lZHUudWNpLmljc - y5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Xb3JrZXJNZXRyaWNzUmVzcG9uc2UShwEKDFJlc3VtZVdvcmtlchI3LmVkd - S51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmVxdWVzdBo+LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ - 2luZS5hcmNoaXRlY3R1cmUucnBjLldvcmtlclN0YXRlUmVzcG9uc2USgAEKDVJldHJpZXZlU3RhdGUSNy5lZHUudWNpLmljcy5hb - WJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJlcXVlc3QaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0Z - WN0dXJlLnJwYy5FbXB0eVJldHVybhKEAQoRUmV0cnlDdXJyZW50VHVwbGUSNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJja - Gl0ZWN0dXJlLnJwYy5FbXB0eVJlcXVlc3QaNi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0e - VJldHVybhKGAQoLU3RhcnRXb3JrZXISNy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FbXB0eVJlc - XVlc3QaPi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Xb3JrZXJTdGF0ZVJlc3BvbnNlEoYBCgxEZ - WJ1Z0NvbW1hbmQSPi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5EZWJ1Z0NvbW1hbmRSZXF1ZXN0G - jYuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRW1wdHlSZXR1cm4SoQEKGEV2YWx1YXRlUHl0aG9uR - XhwcmVzc2lvbhJKLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkV2YWx1YXRlUHl0aG9uRXhwcmVzc - 2lvblJlcXVlc3QaOS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FdmFsdWF0ZWRWYWx1ZRJ+CgtOb - 09wZXJhdGlvbhI3LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmVxdWVzdBo2LmVkdS51Y - 2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLkVtcHR5UmV0dXJuQgniPwZIAFgAeAFiBnByb3RvMw==""" - ).mkString) - lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { - val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) - _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) - } - lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { - val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) - com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlcommandsProto.javaDescriptor, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlreturnsProto.javaDescriptor, - scalapb.options.ScalapbProto.javaDescriptor - )) - } - @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") - def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor -} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/partitionings/Partitioning.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/Partitioning.scala similarity index 64% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/partitionings/Partitioning.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/Partitioning.scala index 704d15fc3c1..910cb833dd3 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/partitionings/Partitioning.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/Partitioning.scala @@ -3,44 +3,44 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings +package edu.uci.ics.amber.engine.architecture.sendsemantics sealed trait Partitioning extends scalapb.GeneratedSealedOneof { - type MessageType = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage - final def isEmpty = this.isInstanceOf[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning.Empty.type] + type MessageType = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage + final def isEmpty = this.isInstanceOf[edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning.Empty.type] final def isDefined = !isEmpty - final def asMessage: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning.PartitioningTypeMapper.toBase(this) - final def asNonEmpty: Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning.NonEmpty] = if (isEmpty) None else Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning.NonEmpty]) + final def asMessage: edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage = edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning.PartitioningTypeMapper.toBase(this) + final def asNonEmpty: Option[edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning.NonEmpty] = if (isEmpty) None else Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning.NonEmpty]) } object Partitioning { - case object Empty extends edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning + case object Empty extends edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning - sealed trait NonEmpty extends edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning - def defaultInstance: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning = Empty + sealed trait NonEmpty extends edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning + def defaultInstance: edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning = Empty - implicit val PartitioningTypeMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning] = new _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning] { - override def toCustom(__base: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage): edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning = __base.sealedValue match { - case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.OneToOnePartitioning => __v.value - case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.RoundRobinPartitioning => __v.value - case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.HashBasedShufflePartitioning => __v.value - case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.RangeBasedShufflePartitioning => __v.value - case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.BroadcastPartitioning => __v.value - case edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.Empty => Empty + implicit val PartitioningTypeMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage, edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning] = new _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage, edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning] { + override def toCustom(__base: edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage): edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning = __base.sealedValue match { + case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.OneToOnePartitioning => __v.value + case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.RoundRobinPartitioning => __v.value + case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.HashBasedShufflePartitioning => __v.value + case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.RangeBasedShufflePartitioning => __v.value + case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.BroadcastPartitioning => __v.value + case edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.Empty => Empty } - override def toBase(__custom: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning): edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage(__custom match { - case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning => edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.OneToOnePartitioning(__v) - case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning => edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.RoundRobinPartitioning(__v) - case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning => edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.HashBasedShufflePartitioning(__v) - case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning => edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.RangeBasedShufflePartitioning(__v) - case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning => edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.BroadcastPartitioning(__v) - case Empty => edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.Empty + override def toBase(__custom: edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning): edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage(__custom match { + case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning => edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.OneToOnePartitioning(__v) + case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning => edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.RoundRobinPartitioning(__v) + case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning => edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.HashBasedShufflePartitioning(__v) + case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning => edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.RangeBasedShufflePartitioning(__v) + case __v: edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning => edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.BroadcastPartitioning(__v) + case Empty => edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.Empty }) } } @SerialVersionUID(0L) final case class PartitioningMessage( - sealedValue: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue + sealedValue: edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[PartitioningMessage] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -109,18 +109,18 @@ final case class PartitioningMessage( __m.writeTo(_output__) }; } - def getOneToOnePartitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning = sealedValue.oneToOnePartitioning.getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning.defaultInstance) - def withOneToOnePartitioning(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning): PartitioningMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.OneToOnePartitioning(__v)) - def getRoundRobinPartitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning = sealedValue.roundRobinPartitioning.getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning.defaultInstance) - def withRoundRobinPartitioning(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning): PartitioningMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.RoundRobinPartitioning(__v)) - def getHashBasedShufflePartitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning = sealedValue.hashBasedShufflePartitioning.getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning.defaultInstance) - def withHashBasedShufflePartitioning(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning): PartitioningMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.HashBasedShufflePartitioning(__v)) - def getRangeBasedShufflePartitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning = sealedValue.rangeBasedShufflePartitioning.getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning.defaultInstance) - def withRangeBasedShufflePartitioning(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning): PartitioningMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.RangeBasedShufflePartitioning(__v)) - def getBroadcastPartitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning = sealedValue.broadcastPartitioning.getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning.defaultInstance) - def withBroadcastPartitioning(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning): PartitioningMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.BroadcastPartitioning(__v)) - def clearSealedValue: PartitioningMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.Empty) - def withSealedValue(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue): PartitioningMessage = copy(sealedValue = __v) + def getOneToOnePartitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning = sealedValue.oneToOnePartitioning.getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning.defaultInstance) + def withOneToOnePartitioning(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning): PartitioningMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.OneToOnePartitioning(__v)) + def getRoundRobinPartitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning = sealedValue.roundRobinPartitioning.getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning.defaultInstance) + def withRoundRobinPartitioning(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning): PartitioningMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.RoundRobinPartitioning(__v)) + def getHashBasedShufflePartitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning = sealedValue.hashBasedShufflePartitioning.getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning.defaultInstance) + def withHashBasedShufflePartitioning(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning): PartitioningMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.HashBasedShufflePartitioning(__v)) + def getRangeBasedShufflePartitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning = sealedValue.rangeBasedShufflePartitioning.getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning.defaultInstance) + def withRangeBasedShufflePartitioning(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning): PartitioningMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.RangeBasedShufflePartitioning(__v)) + def getBroadcastPartitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning = sealedValue.broadcastPartitioning.getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning.defaultInstance) + def withBroadcastPartitioning(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning): PartitioningMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.BroadcastPartitioning(__v)) + def clearSealedValue: PartitioningMessage = copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.Empty) + def withSealedValue(__v: edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue): PartitioningMessage = copy(sealedValue = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => sealedValue.oneToOnePartitioning.orNull @@ -141,47 +141,47 @@ final case class PartitioningMessage( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.type = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage - def toPartitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning.PartitioningTypeMapper.toCustom(this) + def companion: edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.type = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage + def toPartitioning: edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning = edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning.PartitioningTypeMapper.toCustom(this) // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning]) } -object PartitioningMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage = { - var __sealedValue: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.Empty +object PartitioningMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage = { + var __sealedValue: edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.Empty var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.OneToOnePartitioning(__sealedValue.oneToOnePartitioning.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.OneToOnePartitioning(__sealedValue.oneToOnePartitioning.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => - __sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.RoundRobinPartitioning(__sealedValue.roundRobinPartitioning.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.RoundRobinPartitioning(__sealedValue.roundRobinPartitioning.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 26 => - __sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.HashBasedShufflePartitioning(__sealedValue.hashBasedShufflePartitioning.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.HashBasedShufflePartitioning(__sealedValue.hashBasedShufflePartitioning.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 34 => - __sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.RangeBasedShufflePartitioning(__sealedValue.rangeBasedShufflePartitioning.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.RangeBasedShufflePartitioning(__sealedValue.rangeBasedShufflePartitioning.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 42 => - __sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.BroadcastPartitioning(__sealedValue.broadcastPartitioning.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.BroadcastPartitioning(__sealedValue.broadcastPartitioning.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage( + edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage( sealedValue = __sealedValue ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage( - sealedValue = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning]]).map(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.OneToOnePartitioning(_)) - .orElse[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning]]).map(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.RoundRobinPartitioning(_))) - .orElse[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning]]).map(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.HashBasedShufflePartitioning(_))) - .orElse[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning]]).map(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.RangeBasedShufflePartitioning(_))) - .orElse[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(5).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning]]).map(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.BroadcastPartitioning(_))) - .getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.Empty) + edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage( + sealedValue = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning]]).map(edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.OneToOnePartitioning(_)) + .orElse[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning]]).map(edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.RoundRobinPartitioning(_))) + .orElse[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning]]).map(edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.HashBasedShufflePartitioning(_))) + .orElse[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning]]).map(edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.RangeBasedShufflePartitioning(_))) + .orElse[edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(5).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning]]).map(edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.BroadcastPartitioning(_))) + .getOrElse(edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.Empty) ) case _ => throw new RuntimeException("Expected PMessage") } @@ -190,18 +190,18 @@ object PartitioningMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning - case 2 => __out = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning - case 3 => __out = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning - case 4 => __out = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning - case 5 => __out = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning + case 1 => __out = edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning + case 2 => __out = edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning + case 3 => __out = edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning + case 4 => __out = edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning + case 5 => __out = edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage( - sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.Empty + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage( + sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.Empty ) sealed trait SealedValue extends _root_.scalapb.GeneratedOneof { def isEmpty: _root_.scala.Boolean = false @@ -211,15 +211,15 @@ object PartitioningMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics def isHashBasedShufflePartitioning: _root_.scala.Boolean = false def isRangeBasedShufflePartitioning: _root_.scala.Boolean = false def isBroadcastPartitioning: _root_.scala.Boolean = false - def oneToOnePartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning] = _root_.scala.None - def roundRobinPartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning] = _root_.scala.None - def hashBasedShufflePartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning] = _root_.scala.None - def rangeBasedShufflePartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning] = _root_.scala.None - def broadcastPartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning] = _root_.scala.None + def oneToOnePartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning] = _root_.scala.None + def roundRobinPartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning] = _root_.scala.None + def hashBasedShufflePartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning] = _root_.scala.None + def rangeBasedShufflePartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning] = _root_.scala.None + def broadcastPartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning] = _root_.scala.None } object SealedValue { @SerialVersionUID(0L) - case object Empty extends edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue { + case object Empty extends edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue { type ValueType = _root_.scala.Nothing override def isEmpty: _root_.scala.Boolean = true override def isDefined: _root_.scala.Boolean = false @@ -228,57 +228,57 @@ object PartitioningMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics } @SerialVersionUID(0L) - final case class OneToOnePartitioning(value: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning) extends edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning + final case class OneToOnePartitioning(value: edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning) extends edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning override def isOneToOnePartitioning: _root_.scala.Boolean = true - override def oneToOnePartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning] = Some(value) + override def oneToOnePartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning] = Some(value) override def number: _root_.scala.Int = 1 } @SerialVersionUID(0L) - final case class RoundRobinPartitioning(value: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning) extends edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning + final case class RoundRobinPartitioning(value: edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning) extends edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning override def isRoundRobinPartitioning: _root_.scala.Boolean = true - override def roundRobinPartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning] = Some(value) + override def roundRobinPartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning] = Some(value) override def number: _root_.scala.Int = 2 } @SerialVersionUID(0L) - final case class HashBasedShufflePartitioning(value: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning) extends edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning + final case class HashBasedShufflePartitioning(value: edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning) extends edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning override def isHashBasedShufflePartitioning: _root_.scala.Boolean = true - override def hashBasedShufflePartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning] = Some(value) + override def hashBasedShufflePartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning] = Some(value) override def number: _root_.scala.Int = 3 } @SerialVersionUID(0L) - final case class RangeBasedShufflePartitioning(value: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning) extends edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning + final case class RangeBasedShufflePartitioning(value: edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning) extends edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning override def isRangeBasedShufflePartitioning: _root_.scala.Boolean = true - override def rangeBasedShufflePartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning] = Some(value) + override def rangeBasedShufflePartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning] = Some(value) override def number: _root_.scala.Int = 4 } @SerialVersionUID(0L) - final case class BroadcastPartitioning(value: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning) extends edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning + final case class BroadcastPartitioning(value: edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning) extends edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning override def isBroadcastPartitioning: _root_.scala.Boolean = true - override def broadcastPartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning] = Some(value) + override def broadcastPartitioning: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning] = Some(value) override def number: _root_.scala.Int = 5 } } - implicit class PartitioningMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage](_l) { - def oneToOnePartitioning: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning] = field(_.getOneToOnePartitioning)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.OneToOnePartitioning(f_))) - def roundRobinPartitioning: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning] = field(_.getRoundRobinPartitioning)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.RoundRobinPartitioning(f_))) - def hashBasedShufflePartitioning: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning] = field(_.getHashBasedShufflePartitioning)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.HashBasedShufflePartitioning(f_))) - def rangeBasedShufflePartitioning: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning] = field(_.getRangeBasedShufflePartitioning)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.RangeBasedShufflePartitioning(f_))) - def broadcastPartitioning: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning] = field(_.getBroadcastPartitioning)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue.BroadcastPartitioning(f_))) - def sealedValue: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue] = field(_.sealedValue)((c_, f_) => c_.copy(sealedValue = f_)) + implicit class PartitioningMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage](_l) { + def oneToOnePartitioning: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning] = field(_.getOneToOnePartitioning)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.OneToOnePartitioning(f_))) + def roundRobinPartitioning: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning] = field(_.getRoundRobinPartitioning)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.RoundRobinPartitioning(f_))) + def hashBasedShufflePartitioning: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning] = field(_.getHashBasedShufflePartitioning)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.HashBasedShufflePartitioning(f_))) + def rangeBasedShufflePartitioning: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning] = field(_.getRangeBasedShufflePartitioning)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.RangeBasedShufflePartitioning(f_))) + def broadcastPartitioning: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning] = field(_.getBroadcastPartitioning)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue.BroadcastPartitioning(f_))) + def sealedValue: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue] = field(_.sealedValue)((c_, f_) => c_.copy(sealedValue = f_)) } - final val ONETOONEPARTITIONING_FIELD_NUMBER = 1 - final val ROUNDROBINPARTITIONING_FIELD_NUMBER = 2 - final val HASHBASEDSHUFFLEPARTITIONING_FIELD_NUMBER = 3 - final val RANGEBASEDSHUFFLEPARTITIONING_FIELD_NUMBER = 4 - final val BROADCASTPARTITIONING_FIELD_NUMBER = 5 + final val ONE_TO_ONE_PARTITIONING_FIELD_NUMBER = 1 + final val ROUND_ROBIN_PARTITIONING_FIELD_NUMBER = 2 + final val HASH_BASED_SHUFFLE_PARTITIONING_FIELD_NUMBER = 3 + final val RANGE_BASED_SHUFFLE_PARTITIONING_FIELD_NUMBER = 4 + final val BROADCAST_PARTITIONING_FIELD_NUMBER = 5 def of( - sealedValue: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage.SealedValue - ): _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage = _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage( + sealedValue: edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage.SealedValue + ): _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage = _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage( sealedValue ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning]) @@ -287,8 +287,8 @@ object PartitioningMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics @SerialVersionUID(0L) final case class OneToOnePartitioning( batchSize: _root_.scala.Int, - channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning.NonEmpty with scalapb.lenses.Updatable[OneToOnePartitioning] { + channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity] + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning.NonEmpty with scalapb.lenses.Updatable[OneToOnePartitioning] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -331,9 +331,9 @@ final case class OneToOnePartitioning( } def withBatchSize(__v: _root_.scala.Int): OneToOnePartitioning = copy(batchSize = __v) def clearChannels = copy(channels = _root_.scala.Seq.empty) - def addChannels(__vs: edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity *): OneToOnePartitioning = addAllChannels(__vs) - def addAllChannels(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]): OneToOnePartitioning = copy(channels = channels ++ __vs) - def withChannels(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]): OneToOnePartitioning = copy(channels = __v) + def addChannels(__vs: edu.uci.ics.amber.engine.common.ChannelIdentity *): OneToOnePartitioning = addAllChannels(__vs) + def addAllChannels(__vs: Iterable[edu.uci.ics.amber.engine.common.ChannelIdentity]): OneToOnePartitioning = copy(channels = channels ++ __vs) + def withChannels(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]): OneToOnePartitioning = copy(channels = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -351,15 +351,15 @@ final case class OneToOnePartitioning( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning.type = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning + def companion: edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning.type = edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning]) } -object OneToOnePartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning = { +object OneToOnePartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning = { var __batchSize: _root_.scala.Int = 0 - val __channels: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] + val __channels: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ChannelIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ChannelIdentity] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -368,21 +368,21 @@ object OneToOnePartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ic case 8 => __batchSize = _input__.readInt32() case 18 => - __channels += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity](_input__) + __channels += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ChannelIdentity](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning( + edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning( batchSize = __batchSize, channels = __channels.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning( + edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning( batchSize = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Int]).getOrElse(0), - channels = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]]).getOrElse(_root_.scala.Seq.empty) + channels = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } @@ -391,26 +391,26 @@ object OneToOnePartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ic def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity + case 2 => __out = edu.uci.ics.amber.engine.common.ChannelIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning( batchSize = 0, channels = _root_.scala.Seq.empty ) - implicit class OneToOnePartitioningLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning](_l) { + implicit class OneToOnePartitioningLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning](_l) { def batchSize: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.batchSize)((c_, f_) => c_.copy(batchSize = f_)) - def channels: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]] = field(_.channels)((c_, f_) => c_.copy(channels = f_)) + def channels: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]] = field(_.channels)((c_, f_) => c_.copy(channels = f_)) } - final val BATCHSIZE_FIELD_NUMBER = 1 + final val BATCH_SIZE_FIELD_NUMBER = 1 final val CHANNELS_FIELD_NUMBER = 2 def of( batchSize: _root_.scala.Int, - channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] - ): _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning = _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning( + channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity] + ): _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning = _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning( batchSize, channels ) @@ -420,8 +420,8 @@ object OneToOnePartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ic @SerialVersionUID(0L) final case class RoundRobinPartitioning( batchSize: _root_.scala.Int, - channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning.NonEmpty with scalapb.lenses.Updatable[RoundRobinPartitioning] { + channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity] + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning.NonEmpty with scalapb.lenses.Updatable[RoundRobinPartitioning] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -464,9 +464,9 @@ final case class RoundRobinPartitioning( } def withBatchSize(__v: _root_.scala.Int): RoundRobinPartitioning = copy(batchSize = __v) def clearChannels = copy(channels = _root_.scala.Seq.empty) - def addChannels(__vs: edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity *): RoundRobinPartitioning = addAllChannels(__vs) - def addAllChannels(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]): RoundRobinPartitioning = copy(channels = channels ++ __vs) - def withChannels(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]): RoundRobinPartitioning = copy(channels = __v) + def addChannels(__vs: edu.uci.ics.amber.engine.common.ChannelIdentity *): RoundRobinPartitioning = addAllChannels(__vs) + def addAllChannels(__vs: Iterable[edu.uci.ics.amber.engine.common.ChannelIdentity]): RoundRobinPartitioning = copy(channels = channels ++ __vs) + def withChannels(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]): RoundRobinPartitioning = copy(channels = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -484,15 +484,15 @@ final case class RoundRobinPartitioning( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning.type = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning + def companion: edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning.type = edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning]) } -object RoundRobinPartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning = { +object RoundRobinPartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning = { var __batchSize: _root_.scala.Int = 0 - val __channels: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] + val __channels: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ChannelIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ChannelIdentity] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -501,21 +501,21 @@ object RoundRobinPartitioning extends scalapb.GeneratedMessageCompanion[edu.uci. case 8 => __batchSize = _input__.readInt32() case 18 => - __channels += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity](_input__) + __channels += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ChannelIdentity](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning( + edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning( batchSize = __batchSize, channels = __channels.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning( + edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning( batchSize = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Int]).getOrElse(0), - channels = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]]).getOrElse(_root_.scala.Seq.empty) + channels = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } @@ -524,26 +524,26 @@ object RoundRobinPartitioning extends scalapb.GeneratedMessageCompanion[edu.uci. def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity + case 2 => __out = edu.uci.ics.amber.engine.common.ChannelIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning( batchSize = 0, channels = _root_.scala.Seq.empty ) - implicit class RoundRobinPartitioningLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning](_l) { + implicit class RoundRobinPartitioningLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning](_l) { def batchSize: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.batchSize)((c_, f_) => c_.copy(batchSize = f_)) - def channels: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]] = field(_.channels)((c_, f_) => c_.copy(channels = f_)) + def channels: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]] = field(_.channels)((c_, f_) => c_.copy(channels = f_)) } - final val BATCHSIZE_FIELD_NUMBER = 1 + final val BATCH_SIZE_FIELD_NUMBER = 1 final val CHANNELS_FIELD_NUMBER = 2 def of( batchSize: _root_.scala.Int, - channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] - ): _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning = _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning( + channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity] + ): _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning = _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning( batchSize, channels ) @@ -553,9 +553,9 @@ object RoundRobinPartitioning extends scalapb.GeneratedMessageCompanion[edu.uci. @SerialVersionUID(0L) final case class HashBasedShufflePartitioning( batchSize: _root_.scala.Int, - channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity], + channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity], hashAttributeNames: _root_.scala.Seq[_root_.scala.Predef.String] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning.NonEmpty with scalapb.lenses.Updatable[HashBasedShufflePartitioning] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning.NonEmpty with scalapb.lenses.Updatable[HashBasedShufflePartitioning] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -606,9 +606,9 @@ final case class HashBasedShufflePartitioning( } def withBatchSize(__v: _root_.scala.Int): HashBasedShufflePartitioning = copy(batchSize = __v) def clearChannels = copy(channels = _root_.scala.Seq.empty) - def addChannels(__vs: edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity *): HashBasedShufflePartitioning = addAllChannels(__vs) - def addAllChannels(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]): HashBasedShufflePartitioning = copy(channels = channels ++ __vs) - def withChannels(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]): HashBasedShufflePartitioning = copy(channels = __v) + def addChannels(__vs: edu.uci.ics.amber.engine.common.ChannelIdentity *): HashBasedShufflePartitioning = addAllChannels(__vs) + def addAllChannels(__vs: Iterable[edu.uci.ics.amber.engine.common.ChannelIdentity]): HashBasedShufflePartitioning = copy(channels = channels ++ __vs) + def withChannels(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]): HashBasedShufflePartitioning = copy(channels = __v) def clearHashAttributeNames = copy(hashAttributeNames = _root_.scala.Seq.empty) def addHashAttributeNames(__vs: _root_.scala.Predef.String *): HashBasedShufflePartitioning = addAllHashAttributeNames(__vs) def addAllHashAttributeNames(__vs: Iterable[_root_.scala.Predef.String]): HashBasedShufflePartitioning = copy(hashAttributeNames = hashAttributeNames ++ __vs) @@ -632,15 +632,15 @@ final case class HashBasedShufflePartitioning( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning.type = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning + def companion: edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning.type = edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning]) } -object HashBasedShufflePartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning = { +object HashBasedShufflePartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning = { var __batchSize: _root_.scala.Int = 0 - val __channels: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] + val __channels: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ChannelIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ChannelIdentity] val __hashAttributeNames: _root_.scala.collection.immutable.VectorBuilder[_root_.scala.Predef.String] = new _root_.scala.collection.immutable.VectorBuilder[_root_.scala.Predef.String] var _done__ = false while (!_done__) { @@ -650,24 +650,24 @@ object HashBasedShufflePartitioning extends scalapb.GeneratedMessageCompanion[ed case 8 => __batchSize = _input__.readInt32() case 18 => - __channels += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity](_input__) + __channels += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ChannelIdentity](_input__) case 26 => __hashAttributeNames += _input__.readStringRequireUtf8() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning( + edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning( batchSize = __batchSize, channels = __channels.result(), hashAttributeNames = __hashAttributeNames.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning( + edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning( batchSize = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Int]).getOrElse(0), - channels = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]]).getOrElse(_root_.scala.Seq.empty), + channels = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]]).getOrElse(_root_.scala.Seq.empty), hashAttributeNames = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Seq[_root_.scala.Predef.String]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") @@ -677,30 +677,30 @@ object HashBasedShufflePartitioning extends scalapb.GeneratedMessageCompanion[ed def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity + case 2 => __out = edu.uci.ics.amber.engine.common.ChannelIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning( batchSize = 0, channels = _root_.scala.Seq.empty, hashAttributeNames = _root_.scala.Seq.empty ) - implicit class HashBasedShufflePartitioningLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning](_l) { + implicit class HashBasedShufflePartitioningLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning](_l) { def batchSize: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.batchSize)((c_, f_) => c_.copy(batchSize = f_)) - def channels: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]] = field(_.channels)((c_, f_) => c_.copy(channels = f_)) + def channels: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]] = field(_.channels)((c_, f_) => c_.copy(channels = f_)) def hashAttributeNames: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[_root_.scala.Predef.String]] = field(_.hashAttributeNames)((c_, f_) => c_.copy(hashAttributeNames = f_)) } - final val BATCHSIZE_FIELD_NUMBER = 1 + final val BATCH_SIZE_FIELD_NUMBER = 1 final val CHANNELS_FIELD_NUMBER = 2 - final val HASHATTRIBUTENAMES_FIELD_NUMBER = 3 + final val HASH_ATTRIBUTE_NAMES_FIELD_NUMBER = 3 def of( batchSize: _root_.scala.Int, - channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity], + channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity], hashAttributeNames: _root_.scala.Seq[_root_.scala.Predef.String] - ): _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning = _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning( + ): _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning = _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning( batchSize, channels, hashAttributeNames @@ -711,11 +711,11 @@ object HashBasedShufflePartitioning extends scalapb.GeneratedMessageCompanion[ed @SerialVersionUID(0L) final case class RangeBasedShufflePartitioning( batchSize: _root_.scala.Int, - channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity], + channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity], rangeAttributeNames: _root_.scala.Seq[_root_.scala.Predef.String], rangeMin: _root_.scala.Long, rangeMax: _root_.scala.Long - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning.NonEmpty with scalapb.lenses.Updatable[RangeBasedShufflePartitioning] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning.NonEmpty with scalapb.lenses.Updatable[RangeBasedShufflePartitioning] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -792,9 +792,9 @@ final case class RangeBasedShufflePartitioning( } def withBatchSize(__v: _root_.scala.Int): RangeBasedShufflePartitioning = copy(batchSize = __v) def clearChannels = copy(channels = _root_.scala.Seq.empty) - def addChannels(__vs: edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity *): RangeBasedShufflePartitioning = addAllChannels(__vs) - def addAllChannels(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]): RangeBasedShufflePartitioning = copy(channels = channels ++ __vs) - def withChannels(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]): RangeBasedShufflePartitioning = copy(channels = __v) + def addChannels(__vs: edu.uci.ics.amber.engine.common.ChannelIdentity *): RangeBasedShufflePartitioning = addAllChannels(__vs) + def addAllChannels(__vs: Iterable[edu.uci.ics.amber.engine.common.ChannelIdentity]): RangeBasedShufflePartitioning = copy(channels = channels ++ __vs) + def withChannels(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]): RangeBasedShufflePartitioning = copy(channels = __v) def clearRangeAttributeNames = copy(rangeAttributeNames = _root_.scala.Seq.empty) def addRangeAttributeNames(__vs: _root_.scala.Predef.String *): RangeBasedShufflePartitioning = addAllRangeAttributeNames(__vs) def addAllRangeAttributeNames(__vs: Iterable[_root_.scala.Predef.String]): RangeBasedShufflePartitioning = copy(rangeAttributeNames = rangeAttributeNames ++ __vs) @@ -830,15 +830,15 @@ final case class RangeBasedShufflePartitioning( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning.type = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning + def companion: edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning.type = edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning]) } -object RangeBasedShufflePartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning = { +object RangeBasedShufflePartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning = { var __batchSize: _root_.scala.Int = 0 - val __channels: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] + val __channels: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ChannelIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ChannelIdentity] val __rangeAttributeNames: _root_.scala.collection.immutable.VectorBuilder[_root_.scala.Predef.String] = new _root_.scala.collection.immutable.VectorBuilder[_root_.scala.Predef.String] var __rangeMin: _root_.scala.Long = 0L var __rangeMax: _root_.scala.Long = 0L @@ -850,7 +850,7 @@ object RangeBasedShufflePartitioning extends scalapb.GeneratedMessageCompanion[e case 8 => __batchSize = _input__.readInt32() case 18 => - __channels += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity](_input__) + __channels += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ChannelIdentity](_input__) case 26 => __rangeAttributeNames += _input__.readStringRequireUtf8() case 32 => @@ -860,7 +860,7 @@ object RangeBasedShufflePartitioning extends scalapb.GeneratedMessageCompanion[e case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning( + edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning( batchSize = __batchSize, channels = __channels.result(), rangeAttributeNames = __rangeAttributeNames.result(), @@ -868,12 +868,12 @@ object RangeBasedShufflePartitioning extends scalapb.GeneratedMessageCompanion[e rangeMax = __rangeMax ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning( + edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning( batchSize = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Int]).getOrElse(0), - channels = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]]).getOrElse(_root_.scala.Seq.empty), + channels = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]]).getOrElse(_root_.scala.Seq.empty), rangeAttributeNames = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Seq[_root_.scala.Predef.String]]).getOrElse(_root_.scala.Seq.empty), rangeMin = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Long]).getOrElse(0L), rangeMax = __fieldsMap.get(scalaDescriptor.findFieldByNumber(5).get).map(_.as[_root_.scala.Long]).getOrElse(0L) @@ -885,38 +885,38 @@ object RangeBasedShufflePartitioning extends scalapb.GeneratedMessageCompanion[e def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity + case 2 => __out = edu.uci.ics.amber.engine.common.ChannelIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning( batchSize = 0, channels = _root_.scala.Seq.empty, rangeAttributeNames = _root_.scala.Seq.empty, rangeMin = 0L, rangeMax = 0L ) - implicit class RangeBasedShufflePartitioningLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning](_l) { + implicit class RangeBasedShufflePartitioningLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning](_l) { def batchSize: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.batchSize)((c_, f_) => c_.copy(batchSize = f_)) - def channels: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]] = field(_.channels)((c_, f_) => c_.copy(channels = f_)) + def channels: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]] = field(_.channels)((c_, f_) => c_.copy(channels = f_)) def rangeAttributeNames: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[_root_.scala.Predef.String]] = field(_.rangeAttributeNames)((c_, f_) => c_.copy(rangeAttributeNames = f_)) def rangeMin: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.rangeMin)((c_, f_) => c_.copy(rangeMin = f_)) def rangeMax: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.rangeMax)((c_, f_) => c_.copy(rangeMax = f_)) } - final val BATCHSIZE_FIELD_NUMBER = 1 + final val BATCH_SIZE_FIELD_NUMBER = 1 final val CHANNELS_FIELD_NUMBER = 2 - final val RANGEATTRIBUTENAMES_FIELD_NUMBER = 3 - final val RANGEMIN_FIELD_NUMBER = 4 - final val RANGEMAX_FIELD_NUMBER = 5 + final val RANGE_ATTRIBUTE_NAMES_FIELD_NUMBER = 3 + final val RANGE_MIN_FIELD_NUMBER = 4 + final val RANGE_MAX_FIELD_NUMBER = 5 def of( batchSize: _root_.scala.Int, - channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity], + channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity], rangeAttributeNames: _root_.scala.Seq[_root_.scala.Predef.String], rangeMin: _root_.scala.Long, rangeMax: _root_.scala.Long - ): _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning = _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning( + ): _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning = _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning( batchSize, channels, rangeAttributeNames, @@ -929,8 +929,8 @@ object RangeBasedShufflePartitioning extends scalapb.GeneratedMessageCompanion[e @SerialVersionUID(0L) final case class BroadcastPartitioning( batchSize: _root_.scala.Int, - channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.Partitioning.NonEmpty with scalapb.lenses.Updatable[BroadcastPartitioning] { + channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity] + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.architecture.sendsemantics.Partitioning.NonEmpty with scalapb.lenses.Updatable[BroadcastPartitioning] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -973,9 +973,9 @@ final case class BroadcastPartitioning( } def withBatchSize(__v: _root_.scala.Int): BroadcastPartitioning = copy(batchSize = __v) def clearChannels = copy(channels = _root_.scala.Seq.empty) - def addChannels(__vs: edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity *): BroadcastPartitioning = addAllChannels(__vs) - def addAllChannels(__vs: Iterable[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]): BroadcastPartitioning = copy(channels = channels ++ __vs) - def withChannels(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]): BroadcastPartitioning = copy(channels = __v) + def addChannels(__vs: edu.uci.ics.amber.engine.common.ChannelIdentity *): BroadcastPartitioning = addAllChannels(__vs) + def addAllChannels(__vs: Iterable[edu.uci.ics.amber.engine.common.ChannelIdentity]): BroadcastPartitioning = copy(channels = channels ++ __vs) + def withChannels(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]): BroadcastPartitioning = copy(channels = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -993,15 +993,15 @@ final case class BroadcastPartitioning( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning.type = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning + def companion: edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning.type = edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning]) } -object BroadcastPartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning = { +object BroadcastPartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning = { var __batchSize: _root_.scala.Int = 0 - val __channels: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] + val __channels: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ChannelIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.ChannelIdentity] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -1010,21 +1010,21 @@ object BroadcastPartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.i case 8 => __batchSize = _input__.readInt32() case 18 => - __channels += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity](_input__) + __channels += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ChannelIdentity](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning( + edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning( batchSize = __batchSize, channels = __channels.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning( + edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning( batchSize = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Int]).getOrElse(0), - channels = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]]).getOrElse(_root_.scala.Seq.empty) + channels = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } @@ -1033,26 +1033,26 @@ object BroadcastPartitioning extends scalapb.GeneratedMessageCompanion[edu.uci.i def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity + case 2 => __out = edu.uci.ics.amber.engine.common.ChannelIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning( batchSize = 0, channels = _root_.scala.Seq.empty ) - implicit class BroadcastPartitioningLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning](_l) { + implicit class BroadcastPartitioningLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning](_l) { def batchSize: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.batchSize)((c_, f_) => c_.copy(batchSize = f_)) - def channels: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]] = field(_.channels)((c_, f_) => c_.copy(channels = f_)) + def channels: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity]] = field(_.channels)((c_, f_) => c_.copy(channels = f_)) } - final val BATCHSIZE_FIELD_NUMBER = 1 + final val BATCH_SIZE_FIELD_NUMBER = 1 final val CHANNELS_FIELD_NUMBER = 2 def of( batchSize: _root_.scala.Int, - channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] - ): _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning = _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning( + channels: _root_.scala.Seq[edu.uci.ics.amber.engine.common.ChannelIdentity] + ): _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning = _root_.edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning( batchSize, channels ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/PartitioningsProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/PartitioningsProto.scala new file mode 100644 index 00000000000..313708c912f --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/PartitioningsProto.scala @@ -0,0 +1,68 @@ +// Generated by the Scala Plugin for the Protocol Buffer Compiler. +// Do not edit! +// +// Protofile syntax: PROTO3 + +package edu.uci.ics.amber.engine.architecture.sendsemantics + +object PartitioningsProto extends _root_.scalapb.GeneratedFileObject { + lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( + scalapb.options.ScalapbProto, + edu.uci.ics.amber.engine.common.VirtualIdentityProto + ) + lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = + Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( + edu.uci.ics.amber.engine.architecture.sendsemantics.PartitioningMessage, + edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning, + edu.uci.ics.amber.engine.architecture.sendsemantics.RoundRobinPartitioning, + edu.uci.ics.amber.engine.architecture.sendsemantics.HashBasedShufflePartitioning, + edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning, + edu.uci.ics.amber.engine.architecture.sendsemantics.BroadcastPartitioning + ) + private lazy val ProtoBytes: _root_.scala.Array[Byte] = + scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( + """CkdlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3NlbmRzZW1hbnRpY3MvcGFydGl0aW9uaW5ncy5wcm90b + xIzZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5zZW5kc2VtYW50aWNzGhVzY2FsYXBiL3NjYWxhcGIucHJvd + G8aNmVkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9jb21tb24vdmlydHVhbF9pZGVudGl0eS5wcm90byKLBwoMUGFydGl0aW9uaW5nE + p0BChdvbmVfdG9fb25lX3BhcnRpdGlvbmluZxgBIAEoCzJJLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUuc + 2VuZHNlbWFudGljcy5PbmVUb09uZVBhcnRpdGlvbmluZ0IZ4j8WEhRvbmVUb09uZVBhcnRpdGlvbmluZ0gAUhRvbmVUb09uZVBhc + nRpdGlvbmluZxKkAQoYcm91bmRfcm9iaW5fcGFydGl0aW9uaW5nGAIgASgLMksuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY + 2hpdGVjdHVyZS5zZW5kc2VtYW50aWNzLlJvdW5kUm9iaW5QYXJ0aXRpb25pbmdCG+I/GBIWcm91bmRSb2JpblBhcnRpdGlvbmluZ + 0gAUhZyb3VuZFJvYmluUGFydGl0aW9uaW5nEr0BCh9oYXNoX2Jhc2VkX3NodWZmbGVfcGFydGl0aW9uaW5nGAMgASgLMlEuZWR1L + nVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5zZW5kc2VtYW50aWNzLkhhc2hCYXNlZFNodWZmbGVQYXJ0aXRpb25pb + mdCIeI/HhIcaGFzaEJhc2VkU2h1ZmZsZVBhcnRpdGlvbmluZ0gAUhxoYXNoQmFzZWRTaHVmZmxlUGFydGl0aW9uaW5nEsEBCiByY + W5nZV9iYXNlZF9zaHVmZmxlX3BhcnRpdGlvbmluZxgEIAEoCzJSLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1c + mUuc2VuZHNlbWFudGljcy5SYW5nZUJhc2VkU2h1ZmZsZVBhcnRpdGlvbmluZ0Ii4j8fEh1yYW5nZUJhc2VkU2h1ZmZsZVBhcnRpd + GlvbmluZ0gAUh1yYW5nZUJhc2VkU2h1ZmZsZVBhcnRpdGlvbmluZxKfAQoWYnJvYWRjYXN0X3BhcnRpdGlvbmluZxgFIAEoCzJKL + mVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUuc2VuZHNlbWFudGljcy5Ccm9hZGNhc3RQYXJ0aXRpb25pbmdCG + uI/FxIVYnJvYWRjYXN0UGFydGl0aW9uaW5nSABSFWJyb2FkY2FzdFBhcnRpdGlvbmluZ0IOCgxzZWFsZWRfdmFsdWUiogEKFE9uZ + VRvT25lUGFydGl0aW9uaW5nEi0KCmJhdGNoX3NpemUYASABKAVCDuI/CxIJYmF0Y2hTaXplUgliYXRjaFNpemUSWwoIY2hhbm5lb + HMYAiADKAsyMC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkNoYW5uZWxJZGVudGl0eUIN4j8KEghjaGFubmVsc1IIY + 2hhbm5lbHMipAEKFlJvdW5kUm9iaW5QYXJ0aXRpb25pbmcSLQoKYmF0Y2hfc2l6ZRgBIAEoBUIO4j8LEgliYXRjaFNpemVSCWJhd + GNoU2l6ZRJbCghjaGFubmVscxgCIAMoCzIwLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQ2hhbm5lbElkZW50aXR5Q + g3iPwoSCGNoYW5uZWxzUghjaGFubmVscyL1AQocSGFzaEJhc2VkU2h1ZmZsZVBhcnRpdGlvbmluZxItCgpiYXRjaF9zaXplGAEgA + SgFQg7iPwsSCWJhdGNoU2l6ZVIJYmF0Y2hTaXplElsKCGNoYW5uZWxzGAIgAygLMjAuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lL + mNvbW1vbi5DaGFubmVsSWRlbnRpdHlCDeI/ChIIY2hhbm5lbHNSCGNoYW5uZWxzEkkKFGhhc2hfYXR0cmlidXRlX25hbWVzGAMgA + ygJQhfiPxQSEmhhc2hBdHRyaWJ1dGVOYW1lc1ISaGFzaEF0dHJpYnV0ZU5hbWVzItECCh1SYW5nZUJhc2VkU2h1ZmZsZVBhcnRpd + GlvbmluZxItCgpiYXRjaF9zaXplGAEgASgFQg7iPwsSCWJhdGNoU2l6ZVIJYmF0Y2hTaXplElsKCGNoYW5uZWxzGAIgAygLMjAuZ + WR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5DaGFubmVsSWRlbnRpdHlCDeI/ChIIY2hhbm5lbHNSCGNoYW5uZWxzEkwKF + XJhbmdlX2F0dHJpYnV0ZV9uYW1lcxgDIAMoCUIY4j8VEhNyYW5nZUF0dHJpYnV0ZU5hbWVzUhNyYW5nZUF0dHJpYnV0ZU5hbWVzE + ioKCXJhbmdlX21pbhgEIAEoA0IN4j8KEghyYW5nZU1pblIIcmFuZ2VNaW4SKgoJcmFuZ2VfbWF4GAUgASgDQg3iPwoSCHJhbmdlT + WF4UghyYW5nZU1heCKjAQoVQnJvYWRjYXN0UGFydGl0aW9uaW5nEi0KCmJhdGNoX3NpemUYASABKAVCDuI/CxIJYmF0Y2hTaXplU + gliYXRjaFNpemUSWwoIY2hhbm5lbHMYAiADKAsyMC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkNoYW5uZWxJZGVud + Gl0eUIN4j8KEghjaGFubmVsc1IIY2hhbm5lbHNCC+I/CBABSABYAHgBYgZwcm90bzM=""" + ).mkString) + lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { + val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) + _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) + } + lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { + val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) + com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( + scalapb.options.ScalapbProto.javaDescriptor, + edu.uci.ics.amber.engine.common.VirtualIdentityProto.javaDescriptor + )) + } + @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") + def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/partitionings/PartitioningsProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/partitionings/PartitioningsProto.scala deleted file mode 100644 index f22a85de803..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/sendsemantics/partitionings/PartitioningsProto.scala +++ /dev/null @@ -1,68 +0,0 @@ -// Generated by the Scala Plugin for the Protocol Buffer Compiler. -// Do not edit! -// -// Protofile syntax: PROTO3 - -package edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings - -object PartitioningsProto extends _root_.scalapb.GeneratedFileObject { - lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( - scalapb.options.ScalapbProto, - edu.uci.ics.amber.engine.common.virtualidentity.VirtualidentityProto - ) - lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = - Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.PartitioningMessage, - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning, - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RoundRobinPartitioning, - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.HashBasedShufflePartitioning, - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning, - edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.BroadcastPartitioning - ) - private lazy val ProtoBytes: _root_.scala.Array[Byte] = - scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( - """CkdlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3NlbmRzZW1hbnRpY3MvcGFydGl0aW9uaW5ncy5wcm90b - xIzZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5zZW5kc2VtYW50aWNzGhVzY2FsYXBiL3NjYWxhcGIucHJvd - G8aNWVkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9jb21tb24vdmlydHVhbGlkZW50aXR5LnByb3RvIv8GCgxQYXJ0aXRpb25pbmcSm - gEKFG9uZVRvT25lUGFydGl0aW9uaW5nGAEgASgLMkkuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5zZW5kc - 2VtYW50aWNzLk9uZVRvT25lUGFydGl0aW9uaW5nQhniPxYSFG9uZVRvT25lUGFydGl0aW9uaW5nSABSFG9uZVRvT25lUGFydGl0a - W9uaW5nEqIBChZyb3VuZFJvYmluUGFydGl0aW9uaW5nGAIgASgLMksuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjd - HVyZS5zZW5kc2VtYW50aWNzLlJvdW5kUm9iaW5QYXJ0aXRpb25pbmdCG+I/GBIWcm91bmRSb2JpblBhcnRpdGlvbmluZ0gAUhZyb - 3VuZFJvYmluUGFydGl0aW9uaW5nEroBChxoYXNoQmFzZWRTaHVmZmxlUGFydGl0aW9uaW5nGAMgASgLMlEuZWR1LnVjaS5pY3MuY - W1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5zZW5kc2VtYW50aWNzLkhhc2hCYXNlZFNodWZmbGVQYXJ0aXRpb25pbmdCIeI/HhIca - GFzaEJhc2VkU2h1ZmZsZVBhcnRpdGlvbmluZ0gAUhxoYXNoQmFzZWRTaHVmZmxlUGFydGl0aW9uaW5nEr4BCh1yYW5nZUJhc2VkU - 2h1ZmZsZVBhcnRpdGlvbmluZxgEIAEoCzJSLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUuc2VuZHNlbWFud - Gljcy5SYW5nZUJhc2VkU2h1ZmZsZVBhcnRpdGlvbmluZ0Ii4j8fEh1yYW5nZUJhc2VkU2h1ZmZsZVBhcnRpdGlvbmluZ0gAUh1yY - W5nZUJhc2VkU2h1ZmZsZVBhcnRpdGlvbmluZxKeAQoVYnJvYWRjYXN0UGFydGl0aW9uaW5nGAUgASgLMkouZWR1LnVjaS5pY3MuY - W1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5zZW5kc2VtYW50aWNzLkJyb2FkY2FzdFBhcnRpdGlvbmluZ0Ia4j8XEhVicm9hZGNhc - 3RQYXJ0aXRpb25pbmdIAFIVYnJvYWRjYXN0UGFydGl0aW9uaW5nQg4KDHNlYWxlZF92YWx1ZSKhAQoUT25lVG9PbmVQYXJ0aXRpb - 25pbmcSLAoJYmF0Y2hTaXplGAEgASgFQg7iPwsSCWJhdGNoU2l6ZVIJYmF0Y2hTaXplElsKCGNoYW5uZWxzGAIgAygLMjAuZWR1L - nVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5DaGFubmVsSWRlbnRpdHlCDeI/ChIIY2hhbm5lbHNSCGNoYW5uZWxzIqMBChZSb - 3VuZFJvYmluUGFydGl0aW9uaW5nEiwKCWJhdGNoU2l6ZRgBIAEoBUIO4j8LEgliYXRjaFNpemVSCWJhdGNoU2l6ZRJbCghjaGFub - mVscxgCIAMoCzIwLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQ2hhbm5lbElkZW50aXR5Qg3iPwoSCGNoYW5uZWxzU - ghjaGFubmVscyLyAQocSGFzaEJhc2VkU2h1ZmZsZVBhcnRpdGlvbmluZxIsCgliYXRjaFNpemUYASABKAVCDuI/CxIJYmF0Y2hTa - XplUgliYXRjaFNpemUSWwoIY2hhbm5lbHMYAiADKAsyMC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkNoYW5uZWxJZ - GVudGl0eUIN4j8KEghjaGFubmVsc1IIY2hhbm5lbHMSRwoSaGFzaEF0dHJpYnV0ZU5hbWVzGAMgAygJQhfiPxQSEmhhc2hBdHRya - WJ1dGVOYW1lc1ISaGFzaEF0dHJpYnV0ZU5hbWVzIswCCh1SYW5nZUJhc2VkU2h1ZmZsZVBhcnRpdGlvbmluZxIsCgliYXRjaFNpe - mUYASABKAVCDuI/CxIJYmF0Y2hTaXplUgliYXRjaFNpemUSWwoIY2hhbm5lbHMYAiADKAsyMC5lZHUudWNpLmljcy5hbWJlci5lb - mdpbmUuY29tbW9uLkNoYW5uZWxJZGVudGl0eUIN4j8KEghjaGFubmVsc1IIY2hhbm5lbHMSSgoTcmFuZ2VBdHRyaWJ1dGVOYW1lc - xgDIAMoCUIY4j8VEhNyYW5nZUF0dHJpYnV0ZU5hbWVzUhNyYW5nZUF0dHJpYnV0ZU5hbWVzEikKCHJhbmdlTWluGAQgASgDQg3iP - woSCHJhbmdlTWluUghyYW5nZU1pbhIpCghyYW5nZU1heBgFIAEoA0IN4j8KEghyYW5nZU1heFIIcmFuZ2VNYXgiogEKFUJyb2FkY - 2FzdFBhcnRpdGlvbmluZxIsCgliYXRjaFNpemUYASABKAVCDuI/CxIJYmF0Y2hTaXplUgliYXRjaFNpemUSWwoIY2hhbm5lbHMYA - iADKAsyMC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkNoYW5uZWxJZGVudGl0eUIN4j8KEghjaGFubmVsc1IIY2hhb - m5lbHNCCeI/BkgAWAB4AWIGcHJvdG8z""" - ).mkString) - lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { - val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) - _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) - } - lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { - val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) - com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( - scalapb.options.ScalapbProto.javaDescriptor, - edu.uci.ics.amber.engine.common.virtualidentity.VirtualidentityProto.javaDescriptor - )) - } - @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") - def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor -} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/PortTupleCountMapping.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/PortTupleCountMapping.scala similarity index 71% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/PortTupleCountMapping.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/PortTupleCountMapping.scala index 3895efd6186..3f84c5b3df9 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/PortTupleCountMapping.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/PortTupleCountMapping.scala @@ -3,11 +3,11 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.worker.statistics +package edu.uci.ics.amber.engine.architecture.worker @SerialVersionUID(0L) final case class PortTupleCountMapping( - portId: edu.uci.ics.amber.engine.common.workflow.PortIdentity, + portId: edu.uci.ics.amber.engine.common.PortIdentity, tupleCount: _root_.scala.Long ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[PortTupleCountMapping] { @transient @@ -55,13 +55,13 @@ final case class PortTupleCountMapping( } }; } - def withPortId(__v: edu.uci.ics.amber.engine.common.workflow.PortIdentity): PortTupleCountMapping = copy(portId = __v) + def withPortId(__v: edu.uci.ics.amber.engine.common.PortIdentity): PortTupleCountMapping = copy(portId = __v) def withTupleCount(__v: _root_.scala.Long): PortTupleCountMapping = copy(tupleCount = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = portId - if (__t != edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance) __t else null } case 2 => { val __t = tupleCount @@ -77,14 +77,14 @@ final case class PortTupleCountMapping( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping.type = edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping + def companion: edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping.type = edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]) } -object PortTupleCountMapping extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping = { - var __portId: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflow.PortIdentity] = _root_.scala.None +object PortTupleCountMapping extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping = { + var __portId: _root_.scala.Option[edu.uci.ics.amber.engine.common.PortIdentity] = _root_.scala.None var __tupleCount: _root_.scala.Long = 0L var _done__ = false while (!_done__) { @@ -92,22 +92,22 @@ object PortTupleCountMapping extends scalapb.GeneratedMessageCompanion[edu.uci.i _tag__ match { case 0 => _done__ = true case 10 => - __portId = _root_.scala.Some(__portId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflow.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __portId = _root_.scala.Some(__portId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 16 => __tupleCount = _input__.readInt64() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping( - portId = __portId.getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping( + portId = __portId.getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance), tupleCount = __tupleCount ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping( - portId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.workflow.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance), + edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping( + portId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance), tupleCount = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Long]).getOrElse(0L) ) case _ => throw new RuntimeException("Expected PMessage") @@ -117,26 +117,26 @@ object PortTupleCountMapping extends scalapb.GeneratedMessageCompanion[edu.uci.i def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.workflow.PortIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.PortIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping( - portId = edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance, + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping( + portId = edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance, tupleCount = 0L ) - implicit class PortTupleCountMappingLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping](_l) { - def portId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PortIdentity] = field(_.portId)((c_, f_) => c_.copy(portId = f_)) + implicit class PortTupleCountMappingLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping](_l) { + def portId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PortIdentity] = field(_.portId)((c_, f_) => c_.copy(portId = f_)) def tupleCount: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.tupleCount)((c_, f_) => c_.copy(tupleCount = f_)) } final val PORT_ID_FIELD_NUMBER = 1 final val TUPLE_COUNT_FIELD_NUMBER = 2 def of( - portId: edu.uci.ics.amber.engine.common.workflow.PortIdentity, + portId: edu.uci.ics.amber.engine.common.PortIdentity, tupleCount: _root_.scala.Long - ): _root_.edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping = _root_.edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping( + ): _root_.edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping = _root_.edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping( portId, tupleCount ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/StatisticsProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/StatisticsProto.scala similarity index 87% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/StatisticsProto.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/StatisticsProto.scala index fccb81a1c69..e1680471ced 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/StatisticsProto.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/StatisticsProto.scala @@ -3,18 +3,18 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.worker.statistics +package edu.uci.ics.amber.engine.architecture.worker object StatisticsProto extends _root_.scalapb.GeneratedFileObject { lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( - edu.uci.ics.amber.engine.common.workflow.WorkflowProto, + edu.uci.ics.amber.engine.common.WorkflowProto, scalapb.options.ScalapbProto ) lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping, - edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics, - edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics + edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping, + edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics, + edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics ) private lazy val ProtoBytes: _root_.scala.Array[Byte] = scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( @@ -35,7 +35,7 @@ object StatisticsProto extends _root_.scalapb.GeneratedFileObject { mtlci5Xb3JrZXJTdGF0aXN0aWNzQhjiPxUSEHdvcmtlclN0YXRpc3RpY3PwAQFSEHdvcmtlclN0YXRpc3RpY3MqngEKC1dvcmtlc lN0YXRlEiUKDVVOSU5JVElBTElaRUQQABoS4j8PEg1VTklOSVRJQUxJWkVEEhUKBVJFQURZEAEaCuI/BxIFUkVBRFkSGQoHUlVOT klORxACGgziPwkSB1JVTk5JTkcSFwoGUEFVU0VEEAMaC+I/CBIGUEFVU0VEEh0KCUNPTVBMRVRFRBAEGg7iPwsSCUNPTVBMRVRFR - EIJ4j8GSABYAHgBYgZwcm90bzM=""" + EIL4j8IEAFIAFgAeAFiBnByb3RvMw==""" ).mkString) lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) @@ -44,7 +44,7 @@ object StatisticsProto extends _root_.scalapb.GeneratedFileObject { lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( - edu.uci.ics.amber.engine.common.workflow.WorkflowProto.javaDescriptor, + edu.uci.ics.amber.engine.common.WorkflowProto.javaDescriptor, scalapb.options.ScalapbProto.javaDescriptor )) } diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/WorkerMetrics.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/WorkerMetrics.scala similarity index 67% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/WorkerMetrics.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/WorkerMetrics.scala index 80da6c8ca52..99bc5cc8469 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/WorkerMetrics.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/WorkerMetrics.scala @@ -3,12 +3,12 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.worker.statistics +package edu.uci.ics.amber.engine.architecture.worker @SerialVersionUID(0L) final case class WorkerMetrics( - workerState: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState, - workerStatistics: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics + workerState: edu.uci.ics.amber.engine.architecture.worker.WorkerState, + workerStatistics: edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[WorkerMetrics] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -55,8 +55,8 @@ final case class WorkerMetrics( } }; } - def withWorkerState(__v: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState): WorkerMetrics = copy(workerState = __v) - def withWorkerStatistics(__v: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics): WorkerMetrics = copy(workerStatistics = __v) + def withWorkerState(__v: edu.uci.ics.amber.engine.architecture.worker.WorkerState): WorkerMetrics = copy(workerState = __v) + def withWorkerStatistics(__v: edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics): WorkerMetrics = copy(workerStatistics = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -65,7 +65,7 @@ final case class WorkerMetrics( } case 2 => { val __t = workerStatistics - if (__t != edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics.defaultInstance) __t else null } } } @@ -77,38 +77,38 @@ final case class WorkerMetrics( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics.type = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics + def companion: edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics.type = edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics]) } -object WorkerMetrics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics = { - var __workerState: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.UNINITIALIZED - var __workerStatistics: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics] = _root_.scala.None +object WorkerMetrics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics = { + var __workerState: edu.uci.ics.amber.engine.architecture.worker.WorkerState = edu.uci.ics.amber.engine.architecture.worker.WorkerState.UNINITIALIZED + var __workerStatistics: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 8 => - __workerState = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.fromValue(_input__.readEnum()) + __workerState = edu.uci.ics.amber.engine.architecture.worker.WorkerState.fromValue(_input__.readEnum()) case 18 => - __workerStatistics = _root_.scala.Some(__workerStatistics.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __workerStatistics = _root_.scala.Some(__workerStatistics.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics( + edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics( workerState = __workerState, - workerStatistics = __workerStatistics.getOrElse(edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics.defaultInstance) + workerStatistics = __workerStatistics.getOrElse(edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics.defaultInstance) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics( - workerState = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.UNINITIALIZED.scalaValueDescriptor).number), - workerStatistics = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics]).getOrElse(edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics.defaultInstance) + edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics( + workerState = edu.uci.ics.amber.engine.architecture.worker.WorkerState.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.worker.WorkerState.UNINITIALIZED.scalaValueDescriptor).number), + workerStatistics = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics]).getOrElse(edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics.defaultInstance) ) case _ => throw new RuntimeException("Expected PMessage") } @@ -117,30 +117,30 @@ object WorkerMetrics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics + case 2 => __out = edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = { (__fieldNumber: @_root_.scala.unchecked) match { - case 1 => edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState + case 1 => edu.uci.ics.amber.engine.architecture.worker.WorkerState } } - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics( - workerState = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.UNINITIALIZED, - workerStatistics = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics.defaultInstance + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics( + workerState = edu.uci.ics.amber.engine.architecture.worker.WorkerState.UNINITIALIZED, + workerStatistics = edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics.defaultInstance ) - implicit class WorkerMetricsLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics](_l) { - def workerState: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState] = field(_.workerState)((c_, f_) => c_.copy(workerState = f_)) - def workerStatistics: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics] = field(_.workerStatistics)((c_, f_) => c_.copy(workerStatistics = f_)) + implicit class WorkerMetricsLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics](_l) { + def workerState: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.WorkerState] = field(_.workerState)((c_, f_) => c_.copy(workerState = f_)) + def workerStatistics: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics] = field(_.workerStatistics)((c_, f_) => c_.copy(workerStatistics = f_)) } final val WORKER_STATE_FIELD_NUMBER = 1 final val WORKER_STATISTICS_FIELD_NUMBER = 2 def of( - workerState: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState, - workerStatistics: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics - ): _root_.edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics = _root_.edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerMetrics( + workerState: edu.uci.ics.amber.engine.architecture.worker.WorkerState, + workerStatistics: edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics + ): _root_.edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics = _root_.edu.uci.ics.amber.engine.architecture.worker.WorkerMetrics( workerState, workerStatistics ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/WorkerState.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/WorkerState.scala similarity index 87% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/WorkerState.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/WorkerState.scala index 0a4d506f505..031199e0a0c 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/WorkerState.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/WorkerState.scala @@ -3,7 +3,7 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.worker.statistics +package edu.uci.ics.amber.engine.architecture.worker sealed abstract class WorkerState(val value: _root_.scala.Int) extends _root_.scalapb.GeneratedEnum { type EnumType = WorkerState @@ -12,8 +12,8 @@ sealed abstract class WorkerState(val value: _root_.scala.Int) extends _root_.sc def isRunning: _root_.scala.Boolean = false def isPaused: _root_.scala.Boolean = false def isCompleted: _root_.scala.Boolean = false - def companion: _root_.scalapb.GeneratedEnumCompanion[WorkerState] = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState - final def asRecognized: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.Recognized] = if (isUnrecognized) _root_.scala.None else _root_.scala.Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.Recognized]) + def companion: _root_.scalapb.GeneratedEnumCompanion[WorkerState] = edu.uci.ics.amber.engine.architecture.worker.WorkerState + final def asRecognized: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.worker.WorkerState.Recognized] = if (isUnrecognized) _root_.scala.None else _root_.scala.Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.worker.WorkerState.Recognized]) } object WorkerState extends _root_.scalapb.GeneratedEnumCompanion[WorkerState] { diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/WorkerStatistics.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/WorkerStatistics.scala similarity index 75% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/WorkerStatistics.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/WorkerStatistics.scala index 63c2de8b30d..ff06b1b3f3c 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/statistics/WorkerStatistics.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/worker/WorkerStatistics.scala @@ -3,12 +3,12 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.worker.statistics +package edu.uci.ics.amber.engine.architecture.worker @SerialVersionUID(0L) final case class WorkerStatistics( - inputTupleCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping], - outputTupleCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping], + inputTupleCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping], + outputTupleCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping], dataProcessingTime: _root_.scala.Long, controlProcessingTime: _root_.scala.Long, idleTime: _root_.scala.Long @@ -90,13 +90,13 @@ final case class WorkerStatistics( }; } def clearInputTupleCount = copy(inputTupleCount = _root_.scala.Seq.empty) - def addInputTupleCount(__vs: edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping *): WorkerStatistics = addAllInputTupleCount(__vs) - def addAllInputTupleCount(__vs: Iterable[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]): WorkerStatistics = copy(inputTupleCount = inputTupleCount ++ __vs) - def withInputTupleCount(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]): WorkerStatistics = copy(inputTupleCount = __v) + def addInputTupleCount(__vs: edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping *): WorkerStatistics = addAllInputTupleCount(__vs) + def addAllInputTupleCount(__vs: Iterable[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]): WorkerStatistics = copy(inputTupleCount = inputTupleCount ++ __vs) + def withInputTupleCount(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]): WorkerStatistics = copy(inputTupleCount = __v) def clearOutputTupleCount = copy(outputTupleCount = _root_.scala.Seq.empty) - def addOutputTupleCount(__vs: edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping *): WorkerStatistics = addAllOutputTupleCount(__vs) - def addAllOutputTupleCount(__vs: Iterable[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]): WorkerStatistics = copy(outputTupleCount = outputTupleCount ++ __vs) - def withOutputTupleCount(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]): WorkerStatistics = copy(outputTupleCount = __v) + def addOutputTupleCount(__vs: edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping *): WorkerStatistics = addAllOutputTupleCount(__vs) + def addAllOutputTupleCount(__vs: Iterable[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]): WorkerStatistics = copy(outputTupleCount = outputTupleCount ++ __vs) + def withOutputTupleCount(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]): WorkerStatistics = copy(outputTupleCount = __v) def withDataProcessingTime(__v: _root_.scala.Long): WorkerStatistics = copy(dataProcessingTime = __v) def withControlProcessingTime(__v: _root_.scala.Long): WorkerStatistics = copy(controlProcessingTime = __v) def withIdleTime(__v: _root_.scala.Long): WorkerStatistics = copy(idleTime = __v) @@ -129,15 +129,15 @@ final case class WorkerStatistics( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics.type = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics + def companion: edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics.type = edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics]) } -object WorkerStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics = { - val __inputTupleCount: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] - val __outputTupleCount: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] +object WorkerStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics = { + val __inputTupleCount: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] + val __outputTupleCount: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] var __dataProcessingTime: _root_.scala.Long = 0L var __controlProcessingTime: _root_.scala.Long = 0L var __idleTime: _root_.scala.Long = 0L @@ -147,9 +147,9 @@ object WorkerStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.am _tag__ match { case 0 => _done__ = true case 10 => - __inputTupleCount += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping](_input__) + __inputTupleCount += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping](_input__) case 18 => - __outputTupleCount += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping](_input__) + __outputTupleCount += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping](_input__) case 24 => __dataProcessingTime = _input__.readInt64() case 32 => @@ -159,7 +159,7 @@ object WorkerStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.am case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics( + edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics( inputTupleCount = __inputTupleCount.result(), outputTupleCount = __outputTupleCount.result(), dataProcessingTime = __dataProcessingTime, @@ -167,12 +167,12 @@ object WorkerStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.am idleTime = __idleTime ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics( - inputTupleCount = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]]).getOrElse(_root_.scala.Seq.empty), - outputTupleCount = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]]).getOrElse(_root_.scala.Seq.empty), + edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics( + inputTupleCount = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]]).getOrElse(_root_.scala.Seq.empty), + outputTupleCount = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]]).getOrElse(_root_.scala.Seq.empty), dataProcessingTime = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Long]).getOrElse(0L), controlProcessingTime = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Long]).getOrElse(0L), idleTime = __fieldsMap.get(scalaDescriptor.findFieldByNumber(5).get).map(_.as[_root_.scala.Long]).getOrElse(0L) @@ -184,23 +184,23 @@ object WorkerStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.am def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping - case 2 => __out = edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping + case 1 => __out = edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping + case 2 => __out = edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics( + lazy val defaultInstance = edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics( inputTupleCount = _root_.scala.Seq.empty, outputTupleCount = _root_.scala.Seq.empty, dataProcessingTime = 0L, controlProcessingTime = 0L, idleTime = 0L ) - implicit class WorkerStatisticsLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics](_l) { - def inputTupleCount: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]] = field(_.inputTupleCount)((c_, f_) => c_.copy(inputTupleCount = f_)) - def outputTupleCount: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]] = field(_.outputTupleCount)((c_, f_) => c_.copy(outputTupleCount = f_)) + implicit class WorkerStatisticsLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics](_l) { + def inputTupleCount: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]] = field(_.inputTupleCount)((c_, f_) => c_.copy(inputTupleCount = f_)) + def outputTupleCount: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]] = field(_.outputTupleCount)((c_, f_) => c_.copy(outputTupleCount = f_)) def dataProcessingTime: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.dataProcessingTime)((c_, f_) => c_.copy(dataProcessingTime = f_)) def controlProcessingTime: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.controlProcessingTime)((c_, f_) => c_.copy(controlProcessingTime = f_)) def idleTime: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.idleTime)((c_, f_) => c_.copy(idleTime = f_)) @@ -211,12 +211,12 @@ object WorkerStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.am final val CONTROL_PROCESSING_TIME_FIELD_NUMBER = 4 final val IDLE_TIME_FIELD_NUMBER = 5 def of( - inputTupleCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping], - outputTupleCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping], + inputTupleCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping], + outputTupleCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping], dataProcessingTime: _root_.scala.Long, controlProcessingTime: _root_.scala.Long, idleTime: _root_.scala.Long - ): _root_.edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics = _root_.edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics( + ): _root_.edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics = _root_.edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics( inputTupleCount, outputTupleCount, dataProcessingTime, diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/actormessage/ActorCommand.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ActorCommand.scala similarity index 55% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/actormessage/ActorCommand.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ActorCommand.scala index 7fc8b294506..af1f7eea0b7 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/actormessage/ActorCommand.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ActorCommand.scala @@ -3,38 +3,38 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.actormessage +package edu.uci.ics.amber.engine.common sealed trait ActorCommand extends scalapb.GeneratedSealedOneof { - type MessageType = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage - final def isEmpty = this.isInstanceOf[edu.uci.ics.amber.engine.common.actormessage.ActorCommand.Empty.type] + type MessageType = edu.uci.ics.amber.engine.common.ActorCommandMessage + final def isEmpty = this.isInstanceOf[edu.uci.ics.amber.engine.common.ActorCommand.Empty.type] final def isDefined = !isEmpty - final def asMessage: edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage = edu.uci.ics.amber.engine.common.actormessage.ActorCommand.ActorCommandTypeMapper.toBase(this) - final def asNonEmpty: Option[edu.uci.ics.amber.engine.common.actormessage.ActorCommand.NonEmpty] = if (isEmpty) None else Some(this.asInstanceOf[edu.uci.ics.amber.engine.common.actormessage.ActorCommand.NonEmpty]) + final def asMessage: edu.uci.ics.amber.engine.common.ActorCommandMessage = edu.uci.ics.amber.engine.common.ActorCommand.ActorCommandTypeMapper.toBase(this) + final def asNonEmpty: Option[edu.uci.ics.amber.engine.common.ActorCommand.NonEmpty] = if (isEmpty) None else Some(this.asInstanceOf[edu.uci.ics.amber.engine.common.ActorCommand.NonEmpty]) } object ActorCommand { - case object Empty extends edu.uci.ics.amber.engine.common.actormessage.ActorCommand + case object Empty extends edu.uci.ics.amber.engine.common.ActorCommand - sealed trait NonEmpty extends edu.uci.ics.amber.engine.common.actormessage.ActorCommand - def defaultInstance: edu.uci.ics.amber.engine.common.actormessage.ActorCommand = Empty + sealed trait NonEmpty extends edu.uci.ics.amber.engine.common.ActorCommand + def defaultInstance: edu.uci.ics.amber.engine.common.ActorCommand = Empty - implicit val ActorCommandTypeMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage, edu.uci.ics.amber.engine.common.actormessage.ActorCommand] = new _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage, edu.uci.ics.amber.engine.common.actormessage.ActorCommand] { - override def toCustom(__base: edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage): edu.uci.ics.amber.engine.common.actormessage.ActorCommand = __base.sealedValue match { - case __v: edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.Backpressure => __v.value - case __v: edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.CreditUpdate => __v.value - case edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.Empty => Empty + implicit val ActorCommandTypeMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ActorCommandMessage, edu.uci.ics.amber.engine.common.ActorCommand] = new _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ActorCommandMessage, edu.uci.ics.amber.engine.common.ActorCommand] { + override def toCustom(__base: edu.uci.ics.amber.engine.common.ActorCommandMessage): edu.uci.ics.amber.engine.common.ActorCommand = __base.sealedValue match { + case __v: edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.Backpressure => __v.value + case __v: edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.CreditUpdate => __v.value + case edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.Empty => Empty } - override def toBase(__custom: edu.uci.ics.amber.engine.common.actormessage.ActorCommand): edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage(__custom match { - case __v: edu.uci.ics.amber.engine.common.actormessage.Backpressure => edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.Backpressure(__v) - case __v: edu.uci.ics.amber.engine.common.actormessage.CreditUpdate => edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.CreditUpdate(__v) - case Empty => edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.Empty + override def toBase(__custom: edu.uci.ics.amber.engine.common.ActorCommand): edu.uci.ics.amber.engine.common.ActorCommandMessage = edu.uci.ics.amber.engine.common.ActorCommandMessage(__custom match { + case __v: edu.uci.ics.amber.engine.common.Backpressure => edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.Backpressure(__v) + case __v: edu.uci.ics.amber.engine.common.CreditUpdate => edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.CreditUpdate(__v) + case Empty => edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.Empty }) } } @SerialVersionUID(0L) final case class ActorCommandMessage( - sealedValue: edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue + sealedValue: edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ActorCommandMessage] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -73,12 +73,12 @@ final case class ActorCommandMessage( __m.writeTo(_output__) }; } - def getBackpressure: edu.uci.ics.amber.engine.common.actormessage.Backpressure = sealedValue.backpressure.getOrElse(edu.uci.ics.amber.engine.common.actormessage.Backpressure.defaultInstance) - def withBackpressure(__v: edu.uci.ics.amber.engine.common.actormessage.Backpressure): ActorCommandMessage = copy(sealedValue = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.Backpressure(__v)) - def getCreditUpdate: edu.uci.ics.amber.engine.common.actormessage.CreditUpdate = sealedValue.creditUpdate.getOrElse(edu.uci.ics.amber.engine.common.actormessage.CreditUpdate.defaultInstance) - def withCreditUpdate(__v: edu.uci.ics.amber.engine.common.actormessage.CreditUpdate): ActorCommandMessage = copy(sealedValue = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.CreditUpdate(__v)) - def clearSealedValue: ActorCommandMessage = copy(sealedValue = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.Empty) - def withSealedValue(__v: edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue): ActorCommandMessage = copy(sealedValue = __v) + def getBackpressure: edu.uci.ics.amber.engine.common.Backpressure = sealedValue.backpressure.getOrElse(edu.uci.ics.amber.engine.common.Backpressure.defaultInstance) + def withBackpressure(__v: edu.uci.ics.amber.engine.common.Backpressure): ActorCommandMessage = copy(sealedValue = edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.Backpressure(__v)) + def getCreditUpdate: edu.uci.ics.amber.engine.common.CreditUpdate = sealedValue.creditUpdate.getOrElse(edu.uci.ics.amber.engine.common.CreditUpdate.defaultInstance) + def withCreditUpdate(__v: edu.uci.ics.amber.engine.common.CreditUpdate): ActorCommandMessage = copy(sealedValue = edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.CreditUpdate(__v)) + def clearSealedValue: ActorCommandMessage = copy(sealedValue = edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.Empty) + def withSealedValue(__v: edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue): ActorCommandMessage = copy(sealedValue = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => sealedValue.backpressure.orNull @@ -93,67 +93,67 @@ final case class ActorCommandMessage( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.type = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage - def toActorCommand: edu.uci.ics.amber.engine.common.actormessage.ActorCommand = edu.uci.ics.amber.engine.common.actormessage.ActorCommand.ActorCommandTypeMapper.toCustom(this) + def companion: edu.uci.ics.amber.engine.common.ActorCommandMessage.type = edu.uci.ics.amber.engine.common.ActorCommandMessage + def toActorCommand: edu.uci.ics.amber.engine.common.ActorCommand = edu.uci.ics.amber.engine.common.ActorCommand.ActorCommandTypeMapper.toCustom(this) // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ActorCommand]) } -object ActorCommandMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage = { - var __sealedValue: edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.Empty +object ActorCommandMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ActorCommandMessage] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ActorCommandMessage] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ActorCommandMessage = { + var __sealedValue: edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue = edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.Empty var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __sealedValue = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.Backpressure(__sealedValue.backpressure.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.actormessage.Backpressure](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.Backpressure(__sealedValue.backpressure.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.Backpressure](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => - __sealedValue = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.CreditUpdate(__sealedValue.creditUpdate.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.actormessage.CreditUpdate](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __sealedValue = edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.CreditUpdate(__sealedValue.creditUpdate.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.CreditUpdate](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage( + edu.uci.ics.amber.engine.common.ActorCommandMessage( sealedValue = __sealedValue ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ActorCommandMessage] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage( - sealedValue = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.actormessage.Backpressure]]).map(edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.Backpressure(_)) - .orElse[edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.actormessage.CreditUpdate]]).map(edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.CreditUpdate(_))) - .getOrElse(edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.Empty) + edu.uci.ics.amber.engine.common.ActorCommandMessage( + sealedValue = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.Backpressure]]).map(edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.Backpressure(_)) + .orElse[edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue](__fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.CreditUpdate]]).map(edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.CreditUpdate(_))) + .getOrElse(edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.Empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ActormessageProto.javaDescriptor.getMessageTypes().get(2) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ActormessageProto.scalaDescriptor.messages(2) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ActorMessageProto.javaDescriptor.getMessageTypes().get(2) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ActorMessageProto.scalaDescriptor.messages(2) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.actormessage.Backpressure - case 2 => __out = edu.uci.ics.amber.engine.common.actormessage.CreditUpdate + case 1 => __out = edu.uci.ics.amber.engine.common.Backpressure + case 2 => __out = edu.uci.ics.amber.engine.common.CreditUpdate } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage( - sealedValue = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.Empty + lazy val defaultInstance = edu.uci.ics.amber.engine.common.ActorCommandMessage( + sealedValue = edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.Empty ) sealed trait SealedValue extends _root_.scalapb.GeneratedOneof { def isEmpty: _root_.scala.Boolean = false def isDefined: _root_.scala.Boolean = true def isBackpressure: _root_.scala.Boolean = false def isCreditUpdate: _root_.scala.Boolean = false - def backpressure: _root_.scala.Option[edu.uci.ics.amber.engine.common.actormessage.Backpressure] = _root_.scala.None - def creditUpdate: _root_.scala.Option[edu.uci.ics.amber.engine.common.actormessage.CreditUpdate] = _root_.scala.None + def backpressure: _root_.scala.Option[edu.uci.ics.amber.engine.common.Backpressure] = _root_.scala.None + def creditUpdate: _root_.scala.Option[edu.uci.ics.amber.engine.common.CreditUpdate] = _root_.scala.None } object SealedValue { @SerialVersionUID(0L) - case object Empty extends edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue { + case object Empty extends edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue { type ValueType = _root_.scala.Nothing override def isEmpty: _root_.scala.Boolean = true override def isDefined: _root_.scala.Boolean = false @@ -162,30 +162,30 @@ object ActorCommandMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics } @SerialVersionUID(0L) - final case class Backpressure(value: edu.uci.ics.amber.engine.common.actormessage.Backpressure) extends edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.common.actormessage.Backpressure + final case class Backpressure(value: edu.uci.ics.amber.engine.common.Backpressure) extends edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.common.Backpressure override def isBackpressure: _root_.scala.Boolean = true - override def backpressure: _root_.scala.Option[edu.uci.ics.amber.engine.common.actormessage.Backpressure] = Some(value) + override def backpressure: _root_.scala.Option[edu.uci.ics.amber.engine.common.Backpressure] = Some(value) override def number: _root_.scala.Int = 1 } @SerialVersionUID(0L) - final case class CreditUpdate(value: edu.uci.ics.amber.engine.common.actormessage.CreditUpdate) extends edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue { - type ValueType = edu.uci.ics.amber.engine.common.actormessage.CreditUpdate + final case class CreditUpdate(value: edu.uci.ics.amber.engine.common.CreditUpdate) extends edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue { + type ValueType = edu.uci.ics.amber.engine.common.CreditUpdate override def isCreditUpdate: _root_.scala.Boolean = true - override def creditUpdate: _root_.scala.Option[edu.uci.ics.amber.engine.common.actormessage.CreditUpdate] = Some(value) + override def creditUpdate: _root_.scala.Option[edu.uci.ics.amber.engine.common.CreditUpdate] = Some(value) override def number: _root_.scala.Int = 2 } } - implicit class ActorCommandMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage](_l) { - def backpressure: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.actormessage.Backpressure] = field(_.getBackpressure)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.Backpressure(f_))) - def creditUpdate: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.actormessage.CreditUpdate] = field(_.getCreditUpdate)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue.CreditUpdate(f_))) - def sealedValue: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue] = field(_.sealedValue)((c_, f_) => c_.copy(sealedValue = f_)) + implicit class ActorCommandMessageLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ActorCommandMessage]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ActorCommandMessage](_l) { + def backpressure: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.Backpressure] = field(_.getBackpressure)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.Backpressure(f_))) + def creditUpdate: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.CreditUpdate] = field(_.getCreditUpdate)((c_, f_) => c_.copy(sealedValue = edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue.CreditUpdate(f_))) + def sealedValue: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue] = field(_.sealedValue)((c_, f_) => c_.copy(sealedValue = f_)) } final val BACKPRESSURE_FIELD_NUMBER = 1 - final val CREDITUPDATE_FIELD_NUMBER = 2 + final val CREDIT_UPDATE_FIELD_NUMBER = 2 def of( - sealedValue: edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage.SealedValue - ): _root_.edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage = _root_.edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage( + sealedValue: edu.uci.ics.amber.engine.common.ActorCommandMessage.SealedValue + ): _root_.edu.uci.ics.amber.engine.common.ActorCommandMessage = _root_.edu.uci.ics.amber.engine.common.ActorCommandMessage( sealedValue ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ActorCommand]) @@ -194,7 +194,7 @@ object ActorCommandMessage extends scalapb.GeneratedMessageCompanion[edu.uci.ics @SerialVersionUID(0L) final case class Backpressure( enableBackpressure: _root_.scala.Boolean - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.common.actormessage.ActorCommand.NonEmpty with scalapb.lenses.Updatable[Backpressure] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.common.ActorCommand.NonEmpty with scalapb.lenses.Updatable[Backpressure] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { @@ -241,13 +241,13 @@ final case class Backpressure( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.actormessage.Backpressure.type = edu.uci.ics.amber.engine.common.actormessage.Backpressure + def companion: edu.uci.ics.amber.engine.common.Backpressure.type = edu.uci.ics.amber.engine.common.Backpressure // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.Backpressure]) } -object Backpressure extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.actormessage.Backpressure] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.actormessage.Backpressure] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.actormessage.Backpressure = { +object Backpressure extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.Backpressure] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.Backpressure] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.Backpressure = { var __enableBackpressure: _root_.scala.Boolean = false var _done__ = false while (!_done__) { @@ -259,33 +259,33 @@ object Backpressure extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber. case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.actormessage.Backpressure( + edu.uci.ics.amber.engine.common.Backpressure( enableBackpressure = __enableBackpressure ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.actormessage.Backpressure] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.Backpressure] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.actormessage.Backpressure( + edu.uci.ics.amber.engine.common.Backpressure( enableBackpressure = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Boolean]).getOrElse(false) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ActormessageProto.javaDescriptor.getMessageTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ActormessageProto.scalaDescriptor.messages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ActorMessageProto.javaDescriptor.getMessageTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ActorMessageProto.scalaDescriptor.messages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.actormessage.Backpressure( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.Backpressure( enableBackpressure = false ) - implicit class BackpressureLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.actormessage.Backpressure]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.actormessage.Backpressure](_l) { + implicit class BackpressureLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.Backpressure]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.Backpressure](_l) { def enableBackpressure: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Boolean] = field(_.enableBackpressure)((c_, f_) => c_.copy(enableBackpressure = f_)) } - final val ENABLEBACKPRESSURE_FIELD_NUMBER = 1 + final val ENABLE_BACKPRESSURE_FIELD_NUMBER = 1 def of( enableBackpressure: _root_.scala.Boolean - ): _root_.edu.uci.ics.amber.engine.common.actormessage.Backpressure = _root_.edu.uci.ics.amber.engine.common.actormessage.Backpressure( + ): _root_.edu.uci.ics.amber.engine.common.Backpressure = _root_.edu.uci.ics.amber.engine.common.Backpressure( enableBackpressure ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.Backpressure]) @@ -293,20 +293,20 @@ object Backpressure extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber. @SerialVersionUID(0L) final case class CreditUpdate( - ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.common.actormessage.ActorCommand.NonEmpty with scalapb.lenses.Updatable[CreditUpdate] { + ) extends scalapb.GeneratedMessage with edu.uci.ics.amber.engine.common.ActorCommand.NonEmpty with scalapb.lenses.Updatable[CreditUpdate] { final override def serializedSize: _root_.scala.Int = 0 def writeTo(`_output__`: _root_.com.google.protobuf.CodedOutputStream): _root_.scala.Unit = { } def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = throw new MatchError(__fieldNumber) def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = throw new MatchError(__field) def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.actormessage.CreditUpdate.type = edu.uci.ics.amber.engine.common.actormessage.CreditUpdate + def companion: edu.uci.ics.amber.engine.common.CreditUpdate.type = edu.uci.ics.amber.engine.common.CreditUpdate // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.CreditUpdate]) } -object CreditUpdate extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.actormessage.CreditUpdate] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.actormessage.CreditUpdate] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.actormessage.CreditUpdate = { +object CreditUpdate extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.CreditUpdate] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.CreditUpdate] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.CreditUpdate = { var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -315,27 +315,27 @@ object CreditUpdate extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber. case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.actormessage.CreditUpdate( + edu.uci.ics.amber.engine.common.CreditUpdate( ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.actormessage.CreditUpdate] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.CreditUpdate] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.actormessage.CreditUpdate( + edu.uci.ics.amber.engine.common.CreditUpdate( ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ActormessageProto.javaDescriptor.getMessageTypes().get(1) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ActormessageProto.scalaDescriptor.messages(1) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = ActorMessageProto.javaDescriptor.getMessageTypes().get(1) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = ActorMessageProto.scalaDescriptor.messages(1) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.actormessage.CreditUpdate( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.CreditUpdate( ) - implicit class CreditUpdateLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.actormessage.CreditUpdate]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.actormessage.CreditUpdate](_l) { + implicit class CreditUpdateLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.CreditUpdate]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.CreditUpdate](_l) { } def of( - ): _root_.edu.uci.ics.amber.engine.common.actormessage.CreditUpdate = _root_.edu.uci.ics.amber.engine.common.actormessage.CreditUpdate( + ): _root_.edu.uci.ics.amber.engine.common.CreditUpdate = _root_.edu.uci.ics.amber.engine.common.CreditUpdate( ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.CreditUpdate]) } diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/actormessage/ActormessageProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ActorMessageProto.scala similarity index 53% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/actormessage/ActormessageProto.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ActorMessageProto.scala index 9d4002fe3a2..94d4b35b5f3 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/actormessage/ActormessageProto.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ActorMessageProto.scala @@ -3,29 +3,27 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.actormessage +package edu.uci.ics.amber.engine.common -object ActormessageProto extends _root_.scalapb.GeneratedFileObject { +object ActorMessageProto extends _root_.scalapb.GeneratedFileObject { lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( scalapb.options.ScalapbProto ) lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - edu.uci.ics.amber.engine.common.actormessage.Backpressure, - edu.uci.ics.amber.engine.common.actormessage.CreditUpdate, - edu.uci.ics.amber.engine.common.actormessage.ActorCommandMessage, - edu.uci.ics.amber.engine.common.actormessage.PythonActorMessage + edu.uci.ics.amber.engine.common.Backpressure, + edu.uci.ics.amber.engine.common.CreditUpdate, + edu.uci.ics.amber.engine.common.ActorCommandMessage ) private lazy val ProtoBytes: _root_.scala.Array[Byte] = scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( - """CjJlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL2FjdG9ybWVzc2FnZS5wcm90bxIfZWR1LnVjaS5pY3MuYW1iZXIuZ - W5naW5lLmNvbW1vbhoVc2NhbGFwYi9zY2FsYXBiLnByb3RvIlcKDEJhY2twcmVzc3VyZRJHChJlbmFibGVCYWNrcHJlc3N1cmUYA - SABKAhCF+I/FBISZW5hYmxlQmFja3ByZXNzdXJlUhJlbmFibGVCYWNrcHJlc3N1cmUiDgoMQ3JlZGl0VXBkYXRlIu4BCgxBY3Rvc - kNvbW1hbmQSZgoMYmFja3ByZXNzdXJlGAEgASgLMi0uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5CYWNrcHJlc3N1c - mVCEeI/DhIMYmFja3ByZXNzdXJlSABSDGJhY2twcmVzc3VyZRJmCgxjcmVkaXRVcGRhdGUYAiABKAsyLS5lZHUudWNpLmljcy5hb - WJlci5lbmdpbmUuY29tbW9uLkNyZWRpdFVwZGF0ZUIR4j8OEgxjcmVkaXRVcGRhdGVIAFIMY3JlZGl0VXBkYXRlQg4KDHNlYWxlZ - F92YWx1ZSJuChJQeXRob25BY3Rvck1lc3NhZ2USWAoHcGF5bG9hZBgBIAEoCzItLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb - 21tb24uQWN0b3JDb21tYW5kQg/iPwwSB3BheWxvYWTwAQFSB3BheWxvYWRCCeI/BkgAWAB4AWIGcHJvdG8z""" + """CjNlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL2FjdG9yX21lc3NhZ2UucHJvdG8SH2VkdS51Y2kuaWNzLmFtYmVyL + mVuZ2luZS5jb21tb24aFXNjYWxhcGIvc2NhbGFwYi5wcm90byJYCgxCYWNrcHJlc3N1cmUSSAoTZW5hYmxlX2JhY2twcmVzc3VyZ + RgBIAEoCEIX4j8UEhJlbmFibGVCYWNrcHJlc3N1cmVSEmVuYWJsZUJhY2twcmVzc3VyZSIOCgxDcmVkaXRVcGRhdGUi7wEKDEFjd + G9yQ29tbWFuZBJmCgxiYWNrcHJlc3N1cmUYASABKAsyLS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkJhY2twcmVzc + 3VyZUIR4j8OEgxiYWNrcHJlc3N1cmVIAFIMYmFja3ByZXNzdXJlEmcKDWNyZWRpdF91cGRhdGUYAiABKAsyLS5lZHUudWNpLmljc + y5hbWJlci5lbmdpbmUuY29tbW9uLkNyZWRpdFVwZGF0ZUIR4j8OEgxjcmVkaXRVcGRhdGVIAFIMY3JlZGl0VXBkYXRlQg4KDHNlY + WxlZF92YWx1ZUIL4j8IEAFIAFgAeAFiBnByb3RvMw==""" ).mkString) lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/ActorVirtualIdentity.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ActorVirtualIdentity.scala similarity index 75% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/ActorVirtualIdentity.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ActorVirtualIdentity.scala index 51bce383b66..d2a33200117 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/ActorVirtualIdentity.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ActorVirtualIdentity.scala @@ -3,7 +3,7 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.virtualidentity +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class ActorVirtualIdentity( @@ -55,13 +55,13 @@ final case class ActorVirtualIdentity( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.type = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity + def companion: edu.uci.ics.amber.engine.common.ActorVirtualIdentity.type = edu.uci.ics.amber.engine.common.ActorVirtualIdentity // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]) } -object ActorVirtualIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity = { +object ActorVirtualIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ActorVirtualIdentity = { var __name: _root_.scala.Predef.String = "" var _done__ = false while (!_done__) { @@ -73,33 +73,33 @@ object ActorVirtualIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ic case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity( + edu.uci.ics.amber.engine.common.ActorVirtualIdentity( name = __name ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity( + edu.uci.ics.amber.engine.common.ActorVirtualIdentity( name = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualidentityProto.javaDescriptor.getMessageTypes().get(2) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualidentityProto.scalaDescriptor.messages(2) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualIdentityProto.javaDescriptor.getMessageTypes().get(2) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualIdentityProto.scalaDescriptor.messages(2) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.ActorVirtualIdentity( name = "" ) - implicit class ActorVirtualIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_l) { + implicit class ActorVirtualIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ActorVirtualIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_l) { def name: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.name)((c_, f_) => c_.copy(name = f_)) } final val NAME_FIELD_NUMBER = 1 def of( name: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity = _root_.edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity( + ): _root_.edu.uci.ics.amber.engine.common.ActorVirtualIdentity = _root_.edu.uci.ics.amber.engine.common.ActorVirtualIdentity( name ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/BreakpointFault.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/BreakpointFault.scala similarity index 74% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/BreakpointFault.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/BreakpointFault.scala index 5c50e98935f..f72ceb9ebe7 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/BreakpointFault.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/BreakpointFault.scala @@ -3,12 +3,12 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class BreakpointFault( workerName: _root_.scala.Predef.String = "", - faultedTuple: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple] = _root_.scala.None + faultedTuple: _root_.scala.Option[edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple] = _root_.scala.None ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[BreakpointFault] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -51,9 +51,9 @@ final case class BreakpointFault( }; } def withWorkerName(__v: _root_.scala.Predef.String): BreakpointFault = copy(workerName = __v) - def getFaultedTuple: edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple = faultedTuple.getOrElse(edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple.defaultInstance) + def getFaultedTuple: edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple = faultedTuple.getOrElse(edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple.defaultInstance) def clearFaultedTuple: BreakpointFault = copy(faultedTuple = _root_.scala.None) - def withFaultedTuple(__v: edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple): BreakpointFault = copy(faultedTuple = Option(__v)) + def withFaultedTuple(__v: edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple): BreakpointFault = copy(faultedTuple = Option(__v)) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -71,15 +71,15 @@ final case class BreakpointFault( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.type = edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault + def companion: edu.uci.ics.amber.engine.common.BreakpointFault.type = edu.uci.ics.amber.engine.common.BreakpointFault // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.BreakpointFault]) } -object BreakpointFault extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault = { +object BreakpointFault extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.BreakpointFault] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.BreakpointFault] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.BreakpointFault = { var __workerName: _root_.scala.Predef.String = "" - var __faultedTuple: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple] = _root_.scala.None + var __faultedTuple: _root_.scala.Option[edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -88,39 +88,39 @@ object BreakpointFault extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amb case 10 => __workerName = _input__.readStringRequireUtf8() case 18 => - __faultedTuple = Option(__faultedTuple.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __faultedTuple = Option(__faultedTuple.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault( + edu.uci.ics.amber.engine.common.BreakpointFault( workerName = __workerName, faultedTuple = __faultedTuple ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.BreakpointFault] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault( + edu.uci.ics.amber.engine.common.BreakpointFault( workerName = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), - faultedTuple = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple]]) + faultedTuple = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple]]) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowruntimestateProto.javaDescriptor.getMessageTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowruntimestateProto.scalaDescriptor.messages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowRuntimeStateProto.javaDescriptor.getMessageTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowRuntimeStateProto.scalaDescriptor.messages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple + case 2 => __out = edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple + _root_.edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple ) def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.BreakpointFault( workerName = "", faultedTuple = _root_.scala.None ) @@ -209,13 +209,13 @@ object BreakpointFault extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amb } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple.type = edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple + def companion: edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple.type = edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple]) } - object BreakpointTuple extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple = { + object BreakpointTuple extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple = { var __id: _root_.scala.Long = 0L var __isInput: _root_.scala.Boolean = false val __tuple: _root_.scala.collection.immutable.VectorBuilder[_root_.scala.Predef.String] = new _root_.scala.collection.immutable.VectorBuilder[_root_.scala.Predef.String] @@ -233,33 +233,33 @@ object BreakpointFault extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amb case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple( + edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple( id = __id, isInput = __isInput, tuple = __tuple.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple( + edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple( id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Long]).getOrElse(0L), isInput = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Boolean]).getOrElse(false), tuple = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Seq[_root_.scala.Predef.String]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.javaDescriptor.getNestedTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.scalaDescriptor.nestedMessages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.common.BreakpointFault.javaDescriptor.getNestedTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.common.BreakpointFault.scalaDescriptor.nestedMessages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple( id = 0L, isInput = false, tuple = _root_.scala.Seq.empty ) - implicit class BreakpointTupleLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple](_l) { + implicit class BreakpointTupleLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple](_l) { def id: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.id)((c_, f_) => c_.copy(id = f_)) def isInput: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Boolean] = field(_.isInput)((c_, f_) => c_.copy(isInput = f_)) def tuple: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[_root_.scala.Predef.String]] = field(_.tuple)((c_, f_) => c_.copy(tuple = f_)) @@ -271,7 +271,7 @@ object BreakpointFault extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amb id: _root_.scala.Long, isInput: _root_.scala.Boolean, tuple: _root_.scala.Seq[_root_.scala.Predef.String] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple( + ): _root_.edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple = _root_.edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple( id, isInput, tuple @@ -279,17 +279,17 @@ object BreakpointFault extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amb // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple]) } - implicit class BreakpointFaultLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault](_l) { + implicit class BreakpointFaultLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.BreakpointFault]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.BreakpointFault](_l) { def workerName: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.workerName)((c_, f_) => c_.copy(workerName = f_)) - def faultedTuple: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple] = field(_.getFaultedTuple)((c_, f_) => c_.copy(faultedTuple = Option(f_))) - def optionalFaultedTuple: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple]] = field(_.faultedTuple)((c_, f_) => c_.copy(faultedTuple = f_)) + def faultedTuple: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple] = field(_.getFaultedTuple)((c_, f_) => c_.copy(faultedTuple = Option(f_))) + def optionalFaultedTuple: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple]] = field(_.faultedTuple)((c_, f_) => c_.copy(faultedTuple = f_)) } final val WORKER_NAME_FIELD_NUMBER = 1 final val FAULTED_TUPLE_FIELD_NUMBER = 2 def of( workerName: _root_.scala.Predef.String, - faultedTuple: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault.BreakpointTuple] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault( + faultedTuple: _root_.scala.Option[edu.uci.ics.amber.engine.common.BreakpointFault.BreakpointTuple] + ): _root_.edu.uci.ics.amber.engine.common.BreakpointFault = _root_.edu.uci.ics.amber.engine.common.BreakpointFault( workerName, faultedTuple ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/ChannelIdentity.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ChannelIdentity.scala similarity index 62% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/ChannelIdentity.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ChannelIdentity.scala index f5c72494569..ddd1327fd4f 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/ChannelIdentity.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ChannelIdentity.scala @@ -3,12 +3,12 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.virtualidentity +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class ChannelIdentity( - fromWorkerId: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity, - toWorkerId: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity, + fromWorkerId: edu.uci.ics.amber.engine.common.ActorVirtualIdentity, + toWorkerId: edu.uci.ics.amber.engine.common.ActorVirtualIdentity, isControl: _root_.scala.Boolean ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ChannelIdentity] { @transient @@ -71,18 +71,18 @@ final case class ChannelIdentity( } }; } - def withFromWorkerId(__v: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity): ChannelIdentity = copy(fromWorkerId = __v) - def withToWorkerId(__v: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity): ChannelIdentity = copy(toWorkerId = __v) + def withFromWorkerId(__v: edu.uci.ics.amber.engine.common.ActorVirtualIdentity): ChannelIdentity = copy(fromWorkerId = __v) + def withToWorkerId(__v: edu.uci.ics.amber.engine.common.ActorVirtualIdentity): ChannelIdentity = copy(toWorkerId = __v) def withIsControl(__v: _root_.scala.Boolean): ChannelIdentity = copy(isControl = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = fromWorkerId - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) __t else null } case 2 => { val __t = toWorkerId - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance) __t else null } case 3 => { val __t = isControl @@ -99,15 +99,15 @@ final case class ChannelIdentity( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity.type = edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity + def companion: edu.uci.ics.amber.engine.common.ChannelIdentity.type = edu.uci.ics.amber.engine.common.ChannelIdentity // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ChannelIdentity]) } -object ChannelIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity = { - var __fromWorkerId: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = _root_.scala.None - var __toWorkerId: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = _root_.scala.None +object ChannelIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ChannelIdentity] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ChannelIdentity] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ChannelIdentity = { + var __fromWorkerId: _root_.scala.Option[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = _root_.scala.None + var __toWorkerId: _root_.scala.Option[edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = _root_.scala.None var __isControl: _root_.scala.Boolean = false var _done__ = false while (!_done__) { @@ -115,60 +115,60 @@ object ChannelIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amb _tag__ match { case 0 => _done__ = true case 10 => - __fromWorkerId = _root_.scala.Some(__fromWorkerId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __fromWorkerId = _root_.scala.Some(__fromWorkerId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => - __toWorkerId = _root_.scala.Some(__toWorkerId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __toWorkerId = _root_.scala.Some(__toWorkerId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ActorVirtualIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 24 => __isControl = _input__.readBool() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity( - fromWorkerId = __fromWorkerId.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance), - toWorkerId = __toWorkerId.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance), + edu.uci.ics.amber.engine.common.ChannelIdentity( + fromWorkerId = __fromWorkerId.getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance), + toWorkerId = __toWorkerId.getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance), isControl = __isControl ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ChannelIdentity] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity( - fromWorkerId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance), - toWorkerId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance), + edu.uci.ics.amber.engine.common.ChannelIdentity( + fromWorkerId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance), + toWorkerId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.ActorVirtualIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance), isControl = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Boolean]).getOrElse(false) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualidentityProto.javaDescriptor.getMessageTypes().get(3) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualidentityProto.scalaDescriptor.messages(3) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualIdentityProto.javaDescriptor.getMessageTypes().get(3) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualIdentityProto.scalaDescriptor.messages(3) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity - case 2 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity + case 2 => __out = edu.uci.ics.amber.engine.common.ActorVirtualIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity( - fromWorkerId = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance, - toWorkerId = edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity.defaultInstance, + lazy val defaultInstance = edu.uci.ics.amber.engine.common.ChannelIdentity( + fromWorkerId = edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance, + toWorkerId = edu.uci.ics.amber.engine.common.ActorVirtualIdentity.defaultInstance, isControl = false ) - implicit class ChannelIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity](_l) { - def fromWorkerId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = field(_.fromWorkerId)((c_, f_) => c_.copy(fromWorkerId = f_)) - def toWorkerId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity] = field(_.toWorkerId)((c_, f_) => c_.copy(toWorkerId = f_)) + implicit class ChannelIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ChannelIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ChannelIdentity](_l) { + def fromWorkerId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = field(_.fromWorkerId)((c_, f_) => c_.copy(fromWorkerId = f_)) + def toWorkerId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ActorVirtualIdentity] = field(_.toWorkerId)((c_, f_) => c_.copy(toWorkerId = f_)) def isControl: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Boolean] = field(_.isControl)((c_, f_) => c_.copy(isControl = f_)) } - final val FROMWORKERID_FIELD_NUMBER = 1 - final val TOWORKERID_FIELD_NUMBER = 2 - final val ISCONTROL_FIELD_NUMBER = 3 + final val FROM_WORKER_ID_FIELD_NUMBER = 1 + final val TO_WORKER_ID_FIELD_NUMBER = 2 + final val IS_CONTROL_FIELD_NUMBER = 3 def of( - fromWorkerId: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity, - toWorkerId: edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity, + fromWorkerId: edu.uci.ics.amber.engine.common.ActorVirtualIdentity, + toWorkerId: edu.uci.ics.amber.engine.common.ActorVirtualIdentity, isControl: _root_.scala.Boolean - ): _root_.edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity = _root_.edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity( + ): _root_.edu.uci.ics.amber.engine.common.ChannelIdentity = _root_.edu.uci.ics.amber.engine.common.ChannelIdentity( fromWorkerId, toWorkerId, isControl diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/ChannelMarkerIdentity.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ChannelMarkerIdentity.scala similarity index 75% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/ChannelMarkerIdentity.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ChannelMarkerIdentity.scala index 4972ec96149..343af2c19f5 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/ChannelMarkerIdentity.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ChannelMarkerIdentity.scala @@ -3,7 +3,7 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.virtualidentity +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class ChannelMarkerIdentity( @@ -55,13 +55,13 @@ final case class ChannelMarkerIdentity( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity.type = edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity + def companion: edu.uci.ics.amber.engine.common.ChannelMarkerIdentity.type = edu.uci.ics.amber.engine.common.ChannelMarkerIdentity // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity]) } -object ChannelMarkerIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity = { +object ChannelMarkerIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ChannelMarkerIdentity = { var __id: _root_.scala.Predef.String = "" var _done__ = false while (!_done__) { @@ -73,33 +73,33 @@ object ChannelMarkerIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.i case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity( + edu.uci.ics.amber.engine.common.ChannelMarkerIdentity( id = __id ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity( + edu.uci.ics.amber.engine.common.ChannelMarkerIdentity( id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualidentityProto.javaDescriptor.getMessageTypes().get(6) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualidentityProto.scalaDescriptor.messages(6) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualIdentityProto.javaDescriptor.getMessageTypes().get(6) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualIdentityProto.scalaDescriptor.messages(6) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.ChannelMarkerIdentity( id = "" ) - implicit class ChannelMarkerIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity](_l) { + implicit class ChannelMarkerIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ChannelMarkerIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ChannelMarkerIdentity](_l) { def id: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.id)((c_, f_) => c_.copy(id = f_)) } final val ID_FIELD_NUMBER = 1 def of( id: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity = _root_.edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity( + ): _root_.edu.uci.ics.amber.engine.common.ChannelMarkerIdentity = _root_.edu.uci.ics.amber.engine.common.ChannelMarkerIdentity( id ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ChannelMarkerIdentity]) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/EvaluatedValueList.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/EvaluatedValueList.scala similarity index 66% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/EvaluatedValueList.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/EvaluatedValueList.scala index 41eb110550e..f3354926098 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/EvaluatedValueList.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/EvaluatedValueList.scala @@ -3,11 +3,11 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class EvaluatedValueList( - values: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] = _root_.scala.Seq.empty + values: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] = _root_.scala.Seq.empty ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[EvaluatedValueList] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -37,9 +37,9 @@ final case class EvaluatedValueList( }; } def clearValues = copy(values = _root_.scala.Seq.empty) - def addValues(__vs: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue *): EvaluatedValueList = addAllValues(__vs) - def addAllValues(__vs: Iterable[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue]): EvaluatedValueList = copy(values = values ++ __vs) - def withValues(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue]): EvaluatedValueList = copy(values = __v) + def addValues(__vs: edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue *): EvaluatedValueList = addAllValues(__vs) + def addAllValues(__vs: Iterable[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]): EvaluatedValueList = copy(values = values ++ __vs) + def withValues(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]): EvaluatedValueList = copy(values = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => values @@ -52,57 +52,57 @@ final case class EvaluatedValueList( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList.type = edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList + def companion: edu.uci.ics.amber.engine.common.EvaluatedValueList.type = edu.uci.ics.amber.engine.common.EvaluatedValueList // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.EvaluatedValueList]) } -object EvaluatedValueList extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList = { - val __values: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] +object EvaluatedValueList extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.EvaluatedValueList] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.EvaluatedValueList] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.EvaluatedValueList = { + val __values: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __values += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue](_input__) + __values += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList( + edu.uci.ics.amber.engine.common.EvaluatedValueList( values = __values.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.EvaluatedValueList] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList( - values = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue]]).getOrElse(_root_.scala.Seq.empty) + edu.uci.ics.amber.engine.common.EvaluatedValueList( + values = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowruntimestateProto.javaDescriptor.getMessageTypes().get(3) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowruntimestateProto.scalaDescriptor.messages(3) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowRuntimeStateProto.javaDescriptor.getMessageTypes().get(3) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowRuntimeStateProto.scalaDescriptor.messages(3) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue + case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.EvaluatedValueList( values = _root_.scala.Seq.empty ) - implicit class EvaluatedValueListLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList](_l) { - def values: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue]] = field(_.values)((c_, f_) => c_.copy(values = f_)) + implicit class EvaluatedValueListLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.EvaluatedValueList]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.EvaluatedValueList](_l) { + def values: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue]] = field(_.values)((c_, f_) => c_.copy(values = f_)) } final val VALUES_FIELD_NUMBER = 1 def of( - values: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EvaluatedValue] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList( + values: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.EvaluatedValue] + ): _root_.edu.uci.ics.amber.engine.common.EvaluatedValueList = _root_.edu.uci.ics.amber.engine.common.EvaluatedValueList( values ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.EvaluatedValueList]) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/ExecutionBreakpointStore.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionBreakpointStore.scala similarity index 57% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/ExecutionBreakpointStore.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionBreakpointStore.scala index 80e7324bca8..a0987ee37fd 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/ExecutionBreakpointStore.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionBreakpointStore.scala @@ -3,18 +3,18 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class ExecutionBreakpointStore( - operatorInfo: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints] = _root_.scala.collection.immutable.Map.empty + operatorInfo: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints] = _root_.scala.collection.immutable.Map.empty ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ExecutionBreakpointStore] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { var __size = 0 operatorInfo.foreach { __item => - val __value = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore._typemapper_operatorInfo.toBase(__item) + val __value = edu.uci.ics.amber.engine.common.ExecutionBreakpointStore._typemapper_operatorInfo.toBase(__item) __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } __size @@ -30,79 +30,79 @@ final case class ExecutionBreakpointStore( } def writeTo(`_output__`: _root_.com.google.protobuf.CodedOutputStream): _root_.scala.Unit = { operatorInfo.foreach { __v => - val __m = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore._typemapper_operatorInfo.toBase(__v) + val __m = edu.uci.ics.amber.engine.common.ExecutionBreakpointStore._typemapper_operatorInfo.toBase(__v) _output__.writeTag(1, 2) _output__.writeUInt32NoTag(__m.serializedSize) __m.writeTo(_output__) }; } def clearOperatorInfo = copy(operatorInfo = _root_.scala.collection.immutable.Map.empty) - def addOperatorInfo(__vs: (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints) *): ExecutionBreakpointStore = addAllOperatorInfo(__vs) - def addAllOperatorInfo(__vs: Iterable[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints)]): ExecutionBreakpointStore = copy(operatorInfo = operatorInfo ++ __vs) - def withOperatorInfo(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints]): ExecutionBreakpointStore = copy(operatorInfo = __v) + def addOperatorInfo(__vs: (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints) *): ExecutionBreakpointStore = addAllOperatorInfo(__vs) + def addAllOperatorInfo(__vs: Iterable[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints)]): ExecutionBreakpointStore = copy(operatorInfo = operatorInfo ++ __vs) + def withOperatorInfo(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints]): ExecutionBreakpointStore = copy(operatorInfo = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { - case 1 => operatorInfo.iterator.map(edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore._typemapper_operatorInfo.toBase(_)).toSeq + case 1 => operatorInfo.iterator.map(edu.uci.ics.amber.engine.common.ExecutionBreakpointStore._typemapper_operatorInfo.toBase(_)).toSeq } } def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = { _root_.scala.Predef.require(__field.containingMessage eq companion.scalaDescriptor) (__field.number: @_root_.scala.unchecked) match { - case 1 => _root_.scalapb.descriptors.PRepeated(operatorInfo.iterator.map(edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore._typemapper_operatorInfo.toBase(_).toPMessage).toVector) + case 1 => _root_.scalapb.descriptors.PRepeated(operatorInfo.iterator.map(edu.uci.ics.amber.engine.common.ExecutionBreakpointStore._typemapper_operatorInfo.toBase(_).toPMessage).toVector) } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.type = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore + def companion: edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.type = edu.uci.ics.amber.engine.common.ExecutionBreakpointStore // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore]) } -object ExecutionBreakpointStore extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore = { - val __operatorInfo: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints] +object ExecutionBreakpointStore extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ExecutionBreakpointStore = { + val __operatorInfo: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __operatorInfo += edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore._typemapper_operatorInfo.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry](_input__)) + __operatorInfo += edu.uci.ics.amber.engine.common.ExecutionBreakpointStore._typemapper_operatorInfo.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry](_input__)) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore( + edu.uci.ics.amber.engine.common.ExecutionBreakpointStore( operatorInfo = __operatorInfo.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore( - operatorInfo = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore._typemapper_operatorInfo.toCustom(_)).toMap + edu.uci.ics.amber.engine.common.ExecutionBreakpointStore( + operatorInfo = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.common.ExecutionBreakpointStore._typemapper_operatorInfo.toCustom(_)).toMap ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowruntimestateProto.javaDescriptor.getMessageTypes().get(2) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowruntimestateProto.scalaDescriptor.messages(2) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowRuntimeStateProto.javaDescriptor.getMessageTypes().get(2) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowRuntimeStateProto.scalaDescriptor.messages(2) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry + case 1 => __out = edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry + _root_.edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry ) def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.ExecutionBreakpointStore( operatorInfo = _root_.scala.collection.immutable.Map.empty ) @SerialVersionUID(0L) final case class OperatorInfoEntry( key: _root_.scala.Predef.String = "", - value: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints] = _root_.scala.None + value: _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorBreakpoints] = _root_.scala.None ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[OperatorInfoEntry] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -145,9 +145,9 @@ object ExecutionBreakpointStore extends scalapb.GeneratedMessageCompanion[edu.uc }; } def withKey(__v: _root_.scala.Predef.String): OperatorInfoEntry = copy(key = __v) - def getValue: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints = value.getOrElse(edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints.defaultInstance) + def getValue: edu.uci.ics.amber.engine.common.OperatorBreakpoints = value.getOrElse(edu.uci.ics.amber.engine.common.OperatorBreakpoints.defaultInstance) def clearValue: OperatorInfoEntry = copy(value = _root_.scala.None) - def withValue(__v: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints): OperatorInfoEntry = copy(value = Option(__v)) + def withValue(__v: edu.uci.ics.amber.engine.common.OperatorBreakpoints): OperatorInfoEntry = copy(value = Option(__v)) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -165,15 +165,15 @@ object ExecutionBreakpointStore extends scalapb.GeneratedMessageCompanion[edu.uc } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry.type = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry + def companion: edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry.type = edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry]) } - object OperatorInfoEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry = { + object OperatorInfoEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry = { var __key: _root_.scala.Predef.String = "" - var __value: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints] = _root_.scala.None + var __value: _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorBreakpoints] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -182,68 +182,68 @@ object ExecutionBreakpointStore extends scalapb.GeneratedMessageCompanion[edu.uc case 10 => __key = _input__.readStringRequireUtf8() case 18 => - __value = Option(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __value = Option(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.OperatorBreakpoints](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry( + edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry( key = __key, value = __value ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry( + edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry( key = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), - value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints]]) + value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorBreakpoints]]) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.javaDescriptor.getNestedTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.scalaDescriptor.nestedMessages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.javaDescriptor.getNestedTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.scalaDescriptor.nestedMessages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints + case 2 => __out = edu.uci.ics.amber.engine.common.OperatorBreakpoints } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry( key = "", value = _root_.scala.None ) - implicit class OperatorInfoEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry](_l) { + implicit class OperatorInfoEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry](_l) { def key: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.key)((c_, f_) => c_.copy(key = f_)) - def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints] = field(_.getValue)((c_, f_) => c_.copy(value = Option(f_))) - def optionalValue: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints]] = field(_.value)((c_, f_) => c_.copy(value = f_)) + def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OperatorBreakpoints] = field(_.getValue)((c_, f_) => c_.copy(value = Option(f_))) + def optionalValue: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorBreakpoints]] = field(_.value)((c_, f_) => c_.copy(value = f_)) } final val KEY_FIELD_NUMBER = 1 final val VALUE_FIELD_NUMBER = 2 @transient - implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints)] = - _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints)](__m => (__m.key, __m.getValue))(__p => edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry(__p._1, Some(__p._2))) + implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints)] = + _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints)](__m => (__m.key, __m.getValue))(__p => edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry(__p._1, Some(__p._2))) def of( key: _root_.scala.Predef.String, - value: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry( + value: _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorBreakpoints] + ): _root_.edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry = _root_.edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry( key, value ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry]) } - implicit class ExecutionBreakpointStoreLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore](_l) { - def operatorInfo: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints]] = field(_.operatorInfo)((c_, f_) => c_.copy(operatorInfo = f_)) + implicit class ExecutionBreakpointStoreLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionBreakpointStore]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionBreakpointStore](_l) { + def operatorInfo: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints]] = field(_.operatorInfo)((c_, f_) => c_.copy(operatorInfo = f_)) } final val OPERATOR_INFO_FIELD_NUMBER = 1 @transient - private[workflowruntimestate] val _typemapper_operatorInfo: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints)]] + private[common] val _typemapper_operatorInfo: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints)]] def of( - operatorInfo: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore( + operatorInfo: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorBreakpoints] + ): _root_.edu.uci.ics.amber.engine.common.ExecutionBreakpointStore = _root_.edu.uci.ics.amber.engine.common.ExecutionBreakpointStore( operatorInfo ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionBreakpointStore]) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/ExecutionConsoleStore.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionConsoleStore.scala similarity index 57% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/ExecutionConsoleStore.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionConsoleStore.scala index 92c20334161..5a7e17d5ec4 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/ExecutionConsoleStore.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionConsoleStore.scala @@ -3,18 +3,18 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class ExecutionConsoleStore( - operatorConsole: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole] = _root_.scala.collection.immutable.Map.empty + operatorConsole: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole] = _root_.scala.collection.immutable.Map.empty ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ExecutionConsoleStore] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 private[this] def __computeSerializedSize(): _root_.scala.Int = { var __size = 0 operatorConsole.foreach { __item => - val __value = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore._typemapper_operatorConsole.toBase(__item) + val __value = edu.uci.ics.amber.engine.common.ExecutionConsoleStore._typemapper_operatorConsole.toBase(__item) __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } __size @@ -30,79 +30,79 @@ final case class ExecutionConsoleStore( } def writeTo(`_output__`: _root_.com.google.protobuf.CodedOutputStream): _root_.scala.Unit = { operatorConsole.foreach { __v => - val __m = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore._typemapper_operatorConsole.toBase(__v) + val __m = edu.uci.ics.amber.engine.common.ExecutionConsoleStore._typemapper_operatorConsole.toBase(__v) _output__.writeTag(1, 2) _output__.writeUInt32NoTag(__m.serializedSize) __m.writeTo(_output__) }; } def clearOperatorConsole = copy(operatorConsole = _root_.scala.collection.immutable.Map.empty) - def addOperatorConsole(__vs: (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole) *): ExecutionConsoleStore = addAllOperatorConsole(__vs) - def addAllOperatorConsole(__vs: Iterable[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole)]): ExecutionConsoleStore = copy(operatorConsole = operatorConsole ++ __vs) - def withOperatorConsole(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole]): ExecutionConsoleStore = copy(operatorConsole = __v) + def addOperatorConsole(__vs: (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole) *): ExecutionConsoleStore = addAllOperatorConsole(__vs) + def addAllOperatorConsole(__vs: Iterable[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole)]): ExecutionConsoleStore = copy(operatorConsole = operatorConsole ++ __vs) + def withOperatorConsole(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole]): ExecutionConsoleStore = copy(operatorConsole = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { - case 1 => operatorConsole.iterator.map(edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore._typemapper_operatorConsole.toBase(_)).toSeq + case 1 => operatorConsole.iterator.map(edu.uci.ics.amber.engine.common.ExecutionConsoleStore._typemapper_operatorConsole.toBase(_)).toSeq } } def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = { _root_.scala.Predef.require(__field.containingMessage eq companion.scalaDescriptor) (__field.number: @_root_.scala.unchecked) match { - case 1 => _root_.scalapb.descriptors.PRepeated(operatorConsole.iterator.map(edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore._typemapper_operatorConsole.toBase(_).toPMessage).toVector) + case 1 => _root_.scalapb.descriptors.PRepeated(operatorConsole.iterator.map(edu.uci.ics.amber.engine.common.ExecutionConsoleStore._typemapper_operatorConsole.toBase(_).toPMessage).toVector) } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.type = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore + def companion: edu.uci.ics.amber.engine.common.ExecutionConsoleStore.type = edu.uci.ics.amber.engine.common.ExecutionConsoleStore // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ExecutionConsoleStore]) } -object ExecutionConsoleStore extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore = { - val __operatorConsole: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole] +object ExecutionConsoleStore extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionConsoleStore] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionConsoleStore] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ExecutionConsoleStore = { + val __operatorConsole: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __operatorConsole += edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore._typemapper_operatorConsole.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry](_input__)) + __operatorConsole += edu.uci.ics.amber.engine.common.ExecutionConsoleStore._typemapper_operatorConsole.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry](_input__)) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore( + edu.uci.ics.amber.engine.common.ExecutionConsoleStore( operatorConsole = __operatorConsole.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ExecutionConsoleStore] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore( - operatorConsole = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore._typemapper_operatorConsole.toCustom(_)).toMap + edu.uci.ics.amber.engine.common.ExecutionConsoleStore( + operatorConsole = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.common.ExecutionConsoleStore._typemapper_operatorConsole.toCustom(_)).toMap ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowruntimestateProto.javaDescriptor.getMessageTypes().get(5) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowruntimestateProto.scalaDescriptor.messages(5) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowRuntimeStateProto.javaDescriptor.getMessageTypes().get(5) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowRuntimeStateProto.scalaDescriptor.messages(5) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry + case 1 => __out = edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry + _root_.edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry ) def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.ExecutionConsoleStore( operatorConsole = _root_.scala.collection.immutable.Map.empty ) @SerialVersionUID(0L) final case class OperatorConsoleEntry( key: _root_.scala.Predef.String = "", - value: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole] = _root_.scala.None + value: _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorConsole] = _root_.scala.None ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[OperatorConsoleEntry] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -145,9 +145,9 @@ object ExecutionConsoleStore extends scalapb.GeneratedMessageCompanion[edu.uci.i }; } def withKey(__v: _root_.scala.Predef.String): OperatorConsoleEntry = copy(key = __v) - def getValue: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole = value.getOrElse(edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.defaultInstance) + def getValue: edu.uci.ics.amber.engine.common.OperatorConsole = value.getOrElse(edu.uci.ics.amber.engine.common.OperatorConsole.defaultInstance) def clearValue: OperatorConsoleEntry = copy(value = _root_.scala.None) - def withValue(__v: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole): OperatorConsoleEntry = copy(value = Option(__v)) + def withValue(__v: edu.uci.ics.amber.engine.common.OperatorConsole): OperatorConsoleEntry = copy(value = Option(__v)) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -165,15 +165,15 @@ object ExecutionConsoleStore extends scalapb.GeneratedMessageCompanion[edu.uci.i } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry.type = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry + def companion: edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry.type = edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry]) } - object OperatorConsoleEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry = { + object OperatorConsoleEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry = { var __key: _root_.scala.Predef.String = "" - var __value: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole] = _root_.scala.None + var __value: _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorConsole] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -182,68 +182,68 @@ object ExecutionConsoleStore extends scalapb.GeneratedMessageCompanion[edu.uci.i case 10 => __key = _input__.readStringRequireUtf8() case 18 => - __value = Option(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __value = Option(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.OperatorConsole](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry( + edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry( key = __key, value = __value ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry( + edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry( key = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), - value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole]]) + value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorConsole]]) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.javaDescriptor.getNestedTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.scalaDescriptor.nestedMessages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.common.ExecutionConsoleStore.javaDescriptor.getNestedTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.common.ExecutionConsoleStore.scalaDescriptor.nestedMessages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole + case 2 => __out = edu.uci.ics.amber.engine.common.OperatorConsole } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry( key = "", value = _root_.scala.None ) - implicit class OperatorConsoleEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry](_l) { + implicit class OperatorConsoleEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry](_l) { def key: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.key)((c_, f_) => c_.copy(key = f_)) - def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole] = field(_.getValue)((c_, f_) => c_.copy(value = Option(f_))) - def optionalValue: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole]] = field(_.value)((c_, f_) => c_.copy(value = f_)) + def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OperatorConsole] = field(_.getValue)((c_, f_) => c_.copy(value = Option(f_))) + def optionalValue: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorConsole]] = field(_.value)((c_, f_) => c_.copy(value = f_)) } final val KEY_FIELD_NUMBER = 1 final val VALUE_FIELD_NUMBER = 2 @transient - implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole)] = - _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole)](__m => (__m.key, __m.getValue))(__p => edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry(__p._1, Some(__p._2))) + implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole)] = + _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole)](__m => (__m.key, __m.getValue))(__p => edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry(__p._1, Some(__p._2))) def of( key: _root_.scala.Predef.String, - value: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry( + value: _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorConsole] + ): _root_.edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry = _root_.edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry( key, value ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry]) } - implicit class ExecutionConsoleStoreLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore](_l) { - def operatorConsole: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole]] = field(_.operatorConsole)((c_, f_) => c_.copy(operatorConsole = f_)) + implicit class ExecutionConsoleStoreLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionConsoleStore]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionConsoleStore](_l) { + def operatorConsole: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole]] = field(_.operatorConsole)((c_, f_) => c_.copy(operatorConsole = f_)) } final val OPERATOR_CONSOLE_FIELD_NUMBER = 1 @transient - private[workflowruntimestate] val _typemapper_operatorConsole: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore.OperatorConsoleEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole)]] + private[common] val _typemapper_operatorConsole: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ExecutionConsoleStore.OperatorConsoleEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole)]] def of( - operatorConsole: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore( + operatorConsole: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorConsole] + ): _root_.edu.uci.ics.amber.engine.common.ExecutionConsoleStore = _root_.edu.uci.ics.amber.engine.common.ExecutionConsoleStore( operatorConsole ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionConsoleStore]) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/ExecutionIdentity.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionIdentity.scala similarity index 75% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/ExecutionIdentity.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionIdentity.scala index d74e8cf99c2..5e3a9d9248d 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/ExecutionIdentity.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionIdentity.scala @@ -3,7 +3,7 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.virtualidentity +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class ExecutionIdentity( @@ -55,13 +55,13 @@ final case class ExecutionIdentity( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity.type = edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity + def companion: edu.uci.ics.amber.engine.common.ExecutionIdentity.type = edu.uci.ics.amber.engine.common.ExecutionIdentity // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ExecutionIdentity]) } -object ExecutionIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity = { +object ExecutionIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionIdentity] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionIdentity] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ExecutionIdentity = { var __id: _root_.scala.Long = 0L var _done__ = false while (!_done__) { @@ -73,33 +73,33 @@ object ExecutionIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.a case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity( + edu.uci.ics.amber.engine.common.ExecutionIdentity( id = __id ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ExecutionIdentity] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity( + edu.uci.ics.amber.engine.common.ExecutionIdentity( id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Long]).getOrElse(0L) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualidentityProto.javaDescriptor.getMessageTypes().get(1) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualidentityProto.scalaDescriptor.messages(1) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualIdentityProto.javaDescriptor.getMessageTypes().get(1) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualIdentityProto.scalaDescriptor.messages(1) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.ExecutionIdentity( id = 0L ) - implicit class ExecutionIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity](_l) { + implicit class ExecutionIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionIdentity](_l) { def id: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.id)((c_, f_) => c_.copy(id = f_)) } final val ID_FIELD_NUMBER = 1 def of( id: _root_.scala.Long - ): _root_.edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity = _root_.edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity( + ): _root_.edu.uci.ics.amber.engine.common.ExecutionIdentity = _root_.edu.uci.ics.amber.engine.common.ExecutionIdentity( id ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionIdentity]) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/ExecutionMetadataStore.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionMetadataStore.scala similarity index 58% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/ExecutionMetadataStore.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionMetadataStore.scala index 7f804431f4a..e8a9cabd456 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/ExecutionMetadataStore.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionMetadataStore.scala @@ -3,13 +3,13 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class ExecutionMetadataStore( - state: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.UNINITIALIZED, - fatalErrors: _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError] = _root_.scala.Seq.empty, - executionId: edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity = edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity.defaultInstance, + state: edu.uci.ics.amber.engine.common.WorkflowAggregatedState = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.UNINITIALIZED, + fatalErrors: _root_.scala.Seq[edu.uci.ics.amber.engine.common.WorkflowFatalError] = _root_.scala.Seq.empty, + executionId: edu.uci.ics.amber.engine.common.ExecutionIdentity = edu.uci.ics.amber.engine.common.ExecutionIdentity.defaultInstance, isRecovering: _root_.scala.Boolean = false ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ExecutionMetadataStore] { @transient @@ -80,12 +80,12 @@ final case class ExecutionMetadataStore( } }; } - def withState(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState): ExecutionMetadataStore = copy(state = __v) + def withState(__v: edu.uci.ics.amber.engine.common.WorkflowAggregatedState): ExecutionMetadataStore = copy(state = __v) def clearFatalErrors = copy(fatalErrors = _root_.scala.Seq.empty) - def addFatalErrors(__vs: edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError *): ExecutionMetadataStore = addAllFatalErrors(__vs) - def addAllFatalErrors(__vs: Iterable[edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError]): ExecutionMetadataStore = copy(fatalErrors = fatalErrors ++ __vs) - def withFatalErrors(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError]): ExecutionMetadataStore = copy(fatalErrors = __v) - def withExecutionId(__v: edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity): ExecutionMetadataStore = copy(executionId = __v) + def addFatalErrors(__vs: edu.uci.ics.amber.engine.common.WorkflowFatalError *): ExecutionMetadataStore = addAllFatalErrors(__vs) + def addAllFatalErrors(__vs: Iterable[edu.uci.ics.amber.engine.common.WorkflowFatalError]): ExecutionMetadataStore = copy(fatalErrors = fatalErrors ++ __vs) + def withFatalErrors(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.WorkflowFatalError]): ExecutionMetadataStore = copy(fatalErrors = __v) + def withExecutionId(__v: edu.uci.ics.amber.engine.common.ExecutionIdentity): ExecutionMetadataStore = copy(executionId = __v) def withIsRecovering(__v: _root_.scala.Boolean): ExecutionMetadataStore = copy(isRecovering = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { @@ -96,7 +96,7 @@ final case class ExecutionMetadataStore( case 2 => fatalErrors case 3 => { val __t = executionId - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.ExecutionIdentity.defaultInstance) __t else null } case 4 => { val __t = isRecovering @@ -114,16 +114,16 @@ final case class ExecutionMetadataStore( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore.type = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore + def companion: edu.uci.ics.amber.engine.common.ExecutionMetadataStore.type = edu.uci.ics.amber.engine.common.ExecutionMetadataStore // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ExecutionMetadataStore]) } -object ExecutionMetadataStore extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore = { - var __state: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.UNINITIALIZED - val __fatalErrors: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError] - var __executionId: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity] = _root_.scala.None +object ExecutionMetadataStore extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionMetadataStore] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionMetadataStore] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ExecutionMetadataStore = { + var __state: edu.uci.ics.amber.engine.common.WorkflowAggregatedState = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.UNINITIALIZED + val __fatalErrors: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.WorkflowFatalError] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.WorkflowFatalError] + var __executionId: _root_.scala.Option[edu.uci.ics.amber.engine.common.ExecutionIdentity] = _root_.scala.None var __isRecovering: _root_.scala.Boolean = false var _done__ = false while (!_done__) { @@ -131,72 +131,72 @@ object ExecutionMetadataStore extends scalapb.GeneratedMessageCompanion[edu.uci. _tag__ match { case 0 => _done__ = true case 8 => - __state = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.fromValue(_input__.readEnum()) + __state = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.fromValue(_input__.readEnum()) case 18 => - __fatalErrors += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError](_input__) + __fatalErrors += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.WorkflowFatalError](_input__) case 26 => - __executionId = _root_.scala.Some(__executionId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __executionId = _root_.scala.Some(__executionId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ExecutionIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 32 => __isRecovering = _input__.readBool() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore( + edu.uci.ics.amber.engine.common.ExecutionMetadataStore( state = __state, fatalErrors = __fatalErrors.result(), - executionId = __executionId.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity.defaultInstance), + executionId = __executionId.getOrElse(edu.uci.ics.amber.engine.common.ExecutionIdentity.defaultInstance), isRecovering = __isRecovering ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ExecutionMetadataStore] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore( - state = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.UNINITIALIZED.scalaValueDescriptor).number), - fatalErrors = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError]]).getOrElse(_root_.scala.Seq.empty), - executionId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity.defaultInstance), + edu.uci.ics.amber.engine.common.ExecutionMetadataStore( + state = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.common.WorkflowAggregatedState.UNINITIALIZED.scalaValueDescriptor).number), + fatalErrors = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.WorkflowFatalError]]).getOrElse(_root_.scala.Seq.empty), + executionId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[edu.uci.ics.amber.engine.common.ExecutionIdentity]).getOrElse(edu.uci.ics.amber.engine.common.ExecutionIdentity.defaultInstance), isRecovering = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Boolean]).getOrElse(false) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowruntimestateProto.javaDescriptor.getMessageTypes().get(11) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowruntimestateProto.scalaDescriptor.messages(11) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowRuntimeStateProto.javaDescriptor.getMessageTypes().get(9) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowRuntimeStateProto.scalaDescriptor.messages(9) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError - case 3 => __out = edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity + case 2 => __out = edu.uci.ics.amber.engine.common.WorkflowFatalError + case 3 => __out = edu.uci.ics.amber.engine.common.ExecutionIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = { (__fieldNumber: @_root_.scala.unchecked) match { - case 1 => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState + case 1 => edu.uci.ics.amber.engine.common.WorkflowAggregatedState } } - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore( - state = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.UNINITIALIZED, + lazy val defaultInstance = edu.uci.ics.amber.engine.common.ExecutionMetadataStore( + state = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.UNINITIALIZED, fatalErrors = _root_.scala.Seq.empty, - executionId = edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity.defaultInstance, + executionId = edu.uci.ics.amber.engine.common.ExecutionIdentity.defaultInstance, isRecovering = false ) - implicit class ExecutionMetadataStoreLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore](_l) { - def state: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState] = field(_.state)((c_, f_) => c_.copy(state = f_)) - def fatalErrors: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError]] = field(_.fatalErrors)((c_, f_) => c_.copy(fatalErrors = f_)) - def executionId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity] = field(_.executionId)((c_, f_) => c_.copy(executionId = f_)) + implicit class ExecutionMetadataStoreLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionMetadataStore]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionMetadataStore](_l) { + def state: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.WorkflowAggregatedState] = field(_.state)((c_, f_) => c_.copy(state = f_)) + def fatalErrors: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.WorkflowFatalError]] = field(_.fatalErrors)((c_, f_) => c_.copy(fatalErrors = f_)) + def executionId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionIdentity] = field(_.executionId)((c_, f_) => c_.copy(executionId = f_)) def isRecovering: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Boolean] = field(_.isRecovering)((c_, f_) => c_.copy(isRecovering = f_)) } final val STATE_FIELD_NUMBER = 1 final val FATAL_ERRORS_FIELD_NUMBER = 2 - final val EXECUTIONID_FIELD_NUMBER = 3 + final val EXECUTION_ID_FIELD_NUMBER = 3 final val IS_RECOVERING_FIELD_NUMBER = 4 def of( - state: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState, - fatalErrors: _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError], - executionId: edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity, + state: edu.uci.ics.amber.engine.common.WorkflowAggregatedState, + fatalErrors: _root_.scala.Seq[edu.uci.ics.amber.engine.common.WorkflowFatalError], + executionId: edu.uci.ics.amber.engine.common.ExecutionIdentity, isRecovering: _root_.scala.Boolean - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore( + ): _root_.edu.uci.ics.amber.engine.common.ExecutionMetadataStore = _root_.edu.uci.ics.amber.engine.common.ExecutionMetadataStore( state, fatalErrors, executionId, diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/ExecutionStatsStore.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionStatsStore.scala similarity index 55% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/ExecutionStatsStore.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionStatsStore.scala index c9827aebd1c..fcb45d31647 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/ExecutionStatsStore.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ExecutionStatsStore.scala @@ -3,14 +3,14 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class ExecutionStatsStore( - startTimeStamp: _root_.scala.Long = 0L, - endTimeStamp: _root_.scala.Long = 0L, - operatorInfo: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics] = _root_.scala.collection.immutable.Map.empty, - operatorWorkerMapping: _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping] = _root_.scala.Seq.empty + startTimestamp: _root_.scala.Long = 0L, + endTimestamp: _root_.scala.Long = 0L, + operatorInfo: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics] = _root_.scala.collection.immutable.Map.empty, + operatorWorkerMapping: _root_.scala.Seq[edu.uci.ics.amber.engine.common.OperatorWorkerMapping] = _root_.scala.Seq.empty ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[ExecutionStatsStore] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -18,20 +18,20 @@ final case class ExecutionStatsStore( var __size = 0 { - val __value = startTimeStamp + val __value = startTimestamp if (__value != 0L) { __size += _root_.com.google.protobuf.CodedOutputStream.computeInt64Size(1, __value) } }; { - val __value = endTimeStamp + val __value = endTimestamp if (__value != 0L) { __size += _root_.com.google.protobuf.CodedOutputStream.computeInt64Size(2, __value) } }; operatorInfo.foreach { __item => - val __value = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore._typemapper_operatorInfo.toBase(__item) + val __value = edu.uci.ics.amber.engine.common.ExecutionStatsStore._typemapper_operatorInfo.toBase(__item) __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } operatorWorkerMapping.foreach { __item => @@ -51,19 +51,19 @@ final case class ExecutionStatsStore( } def writeTo(`_output__`: _root_.com.google.protobuf.CodedOutputStream): _root_.scala.Unit = { { - val __v = startTimeStamp + val __v = startTimestamp if (__v != 0L) { _output__.writeInt64(1, __v) } }; { - val __v = endTimeStamp + val __v = endTimestamp if (__v != 0L) { _output__.writeInt64(2, __v) } }; operatorInfo.foreach { __v => - val __m = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore._typemapper_operatorInfo.toBase(__v) + val __m = edu.uci.ics.amber.engine.common.ExecutionStatsStore._typemapper_operatorInfo.toBase(__v) _output__.writeTag(3, 2) _output__.writeUInt32NoTag(__m.serializedSize) __m.writeTo(_output__) @@ -75,110 +75,110 @@ final case class ExecutionStatsStore( __m.writeTo(_output__) }; } - def withStartTimeStamp(__v: _root_.scala.Long): ExecutionStatsStore = copy(startTimeStamp = __v) - def withEndTimeStamp(__v: _root_.scala.Long): ExecutionStatsStore = copy(endTimeStamp = __v) + def withStartTimestamp(__v: _root_.scala.Long): ExecutionStatsStore = copy(startTimestamp = __v) + def withEndTimestamp(__v: _root_.scala.Long): ExecutionStatsStore = copy(endTimestamp = __v) def clearOperatorInfo = copy(operatorInfo = _root_.scala.collection.immutable.Map.empty) - def addOperatorInfo(__vs: (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics) *): ExecutionStatsStore = addAllOperatorInfo(__vs) - def addAllOperatorInfo(__vs: Iterable[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics)]): ExecutionStatsStore = copy(operatorInfo = operatorInfo ++ __vs) - def withOperatorInfo(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics]): ExecutionStatsStore = copy(operatorInfo = __v) + def addOperatorInfo(__vs: (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics) *): ExecutionStatsStore = addAllOperatorInfo(__vs) + def addAllOperatorInfo(__vs: Iterable[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics)]): ExecutionStatsStore = copy(operatorInfo = operatorInfo ++ __vs) + def withOperatorInfo(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics]): ExecutionStatsStore = copy(operatorInfo = __v) def clearOperatorWorkerMapping = copy(operatorWorkerMapping = _root_.scala.Seq.empty) - def addOperatorWorkerMapping(__vs: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping *): ExecutionStatsStore = addAllOperatorWorkerMapping(__vs) - def addAllOperatorWorkerMapping(__vs: Iterable[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping]): ExecutionStatsStore = copy(operatorWorkerMapping = operatorWorkerMapping ++ __vs) - def withOperatorWorkerMapping(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping]): ExecutionStatsStore = copy(operatorWorkerMapping = __v) + def addOperatorWorkerMapping(__vs: edu.uci.ics.amber.engine.common.OperatorWorkerMapping *): ExecutionStatsStore = addAllOperatorWorkerMapping(__vs) + def addAllOperatorWorkerMapping(__vs: Iterable[edu.uci.ics.amber.engine.common.OperatorWorkerMapping]): ExecutionStatsStore = copy(operatorWorkerMapping = operatorWorkerMapping ++ __vs) + def withOperatorWorkerMapping(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.OperatorWorkerMapping]): ExecutionStatsStore = copy(operatorWorkerMapping = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { - val __t = startTimeStamp + val __t = startTimestamp if (__t != 0L) __t else null } case 2 => { - val __t = endTimeStamp + val __t = endTimestamp if (__t != 0L) __t else null } - case 3 => operatorInfo.iterator.map(edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore._typemapper_operatorInfo.toBase(_)).toSeq + case 3 => operatorInfo.iterator.map(edu.uci.ics.amber.engine.common.ExecutionStatsStore._typemapper_operatorInfo.toBase(_)).toSeq case 4 => operatorWorkerMapping } } def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = { _root_.scala.Predef.require(__field.containingMessage eq companion.scalaDescriptor) (__field.number: @_root_.scala.unchecked) match { - case 1 => _root_.scalapb.descriptors.PLong(startTimeStamp) - case 2 => _root_.scalapb.descriptors.PLong(endTimeStamp) - case 3 => _root_.scalapb.descriptors.PRepeated(operatorInfo.iterator.map(edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore._typemapper_operatorInfo.toBase(_).toPMessage).toVector) + case 1 => _root_.scalapb.descriptors.PLong(startTimestamp) + case 2 => _root_.scalapb.descriptors.PLong(endTimestamp) + case 3 => _root_.scalapb.descriptors.PRepeated(operatorInfo.iterator.map(edu.uci.ics.amber.engine.common.ExecutionStatsStore._typemapper_operatorInfo.toBase(_).toPMessage).toVector) case 4 => _root_.scalapb.descriptors.PRepeated(operatorWorkerMapping.iterator.map(_.toPMessage).toVector) } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.type = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore + def companion: edu.uci.ics.amber.engine.common.ExecutionStatsStore.type = edu.uci.ics.amber.engine.common.ExecutionStatsStore // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ExecutionStatsStore]) } -object ExecutionStatsStore extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore = { - var __startTimeStamp: _root_.scala.Long = 0L - var __endTimeStamp: _root_.scala.Long = 0L - val __operatorInfo: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics] - val __operatorWorkerMapping: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping] +object ExecutionStatsStore extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionStatsStore] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionStatsStore] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ExecutionStatsStore = { + var __startTimestamp: _root_.scala.Long = 0L + var __endTimestamp: _root_.scala.Long = 0L + val __operatorInfo: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics] + val __operatorWorkerMapping: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.OperatorWorkerMapping] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.OperatorWorkerMapping] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 8 => - __startTimeStamp = _input__.readInt64() + __startTimestamp = _input__.readInt64() case 16 => - __endTimeStamp = _input__.readInt64() + __endTimestamp = _input__.readInt64() case 26 => - __operatorInfo += edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore._typemapper_operatorInfo.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry](_input__)) + __operatorInfo += edu.uci.ics.amber.engine.common.ExecutionStatsStore._typemapper_operatorInfo.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry](_input__)) case 34 => - __operatorWorkerMapping += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping](_input__) + __operatorWorkerMapping += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.OperatorWorkerMapping](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore( - startTimeStamp = __startTimeStamp, - endTimeStamp = __endTimeStamp, + edu.uci.ics.amber.engine.common.ExecutionStatsStore( + startTimestamp = __startTimestamp, + endTimestamp = __endTimestamp, operatorInfo = __operatorInfo.result(), operatorWorkerMapping = __operatorWorkerMapping.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ExecutionStatsStore] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore( - startTimeStamp = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Long]).getOrElse(0L), - endTimeStamp = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Long]).getOrElse(0L), - operatorInfo = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore._typemapper_operatorInfo.toCustom(_)).toMap, - operatorWorkerMapping = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping]]).getOrElse(_root_.scala.Seq.empty) + edu.uci.ics.amber.engine.common.ExecutionStatsStore( + startTimestamp = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Long]).getOrElse(0L), + endTimestamp = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Long]).getOrElse(0L), + operatorInfo = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.common.ExecutionStatsStore._typemapper_operatorInfo.toCustom(_)).toMap, + operatorWorkerMapping = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.OperatorWorkerMapping]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowruntimestateProto.javaDescriptor.getMessageTypes().get(9) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowruntimestateProto.scalaDescriptor.messages(9) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowRuntimeStateProto.javaDescriptor.getMessageTypes().get(7) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowRuntimeStateProto.scalaDescriptor.messages(7) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 3 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry - case 4 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping + case 3 => __out = edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry + case 4 => __out = edu.uci.ics.amber.engine.common.OperatorWorkerMapping } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry + _root_.edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry ) def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore( - startTimeStamp = 0L, - endTimeStamp = 0L, + lazy val defaultInstance = edu.uci.ics.amber.engine.common.ExecutionStatsStore( + startTimestamp = 0L, + endTimestamp = 0L, operatorInfo = _root_.scala.collection.immutable.Map.empty, operatorWorkerMapping = _root_.scala.Seq.empty ) @SerialVersionUID(0L) final case class OperatorInfoEntry( key: _root_.scala.Predef.String = "", - value: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics] = _root_.scala.None + value: _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorMetrics] = _root_.scala.None ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[OperatorInfoEntry] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -221,9 +221,9 @@ object ExecutionStatsStore extends scalapb.GeneratedMessageCompanion[edu.uci.ics }; } def withKey(__v: _root_.scala.Predef.String): OperatorInfoEntry = copy(key = __v) - def getValue: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics = value.getOrElse(edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics.defaultInstance) + def getValue: edu.uci.ics.amber.engine.common.OperatorMetrics = value.getOrElse(edu.uci.ics.amber.engine.common.OperatorMetrics.defaultInstance) def clearValue: OperatorInfoEntry = copy(value = _root_.scala.None) - def withValue(__v: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics): OperatorInfoEntry = copy(value = Option(__v)) + def withValue(__v: edu.uci.ics.amber.engine.common.OperatorMetrics): OperatorInfoEntry = copy(value = Option(__v)) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -241,15 +241,15 @@ object ExecutionStatsStore extends scalapb.GeneratedMessageCompanion[edu.uci.ics } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry.type = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry + def companion: edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry.type = edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry]) } - object OperatorInfoEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry = { + object OperatorInfoEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry = { var __key: _root_.scala.Predef.String = "" - var __value: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics] = _root_.scala.None + var __value: _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorMetrics] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -258,79 +258,79 @@ object ExecutionStatsStore extends scalapb.GeneratedMessageCompanion[edu.uci.ics case 10 => __key = _input__.readStringRequireUtf8() case 18 => - __value = Option(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __value = Option(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.OperatorMetrics](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry( + edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry( key = __key, value = __value ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry( + edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry( key = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), - value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics]]) + value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorMetrics]]) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.javaDescriptor.getNestedTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.scalaDescriptor.nestedMessages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.common.ExecutionStatsStore.javaDescriptor.getNestedTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.common.ExecutionStatsStore.scalaDescriptor.nestedMessages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics + case 2 => __out = edu.uci.ics.amber.engine.common.OperatorMetrics } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry( key = "", value = _root_.scala.None ) - implicit class OperatorInfoEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry](_l) { + implicit class OperatorInfoEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry](_l) { def key: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.key)((c_, f_) => c_.copy(key = f_)) - def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics] = field(_.getValue)((c_, f_) => c_.copy(value = Option(f_))) - def optionalValue: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics]] = field(_.value)((c_, f_) => c_.copy(value = f_)) + def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OperatorMetrics] = field(_.getValue)((c_, f_) => c_.copy(value = Option(f_))) + def optionalValue: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorMetrics]] = field(_.value)((c_, f_) => c_.copy(value = f_)) } final val KEY_FIELD_NUMBER = 1 final val VALUE_FIELD_NUMBER = 2 @transient - implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics)] = - _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics)](__m => (__m.key, __m.getValue))(__p => edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry(__p._1, Some(__p._2))) + implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics)] = + _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics)](__m => (__m.key, __m.getValue))(__p => edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry(__p._1, Some(__p._2))) def of( key: _root_.scala.Predef.String, - value: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry( + value: _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorMetrics] + ): _root_.edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry = _root_.edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry( key, value ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry]) } - implicit class ExecutionStatsStoreLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore](_l) { - def startTimeStamp: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.startTimeStamp)((c_, f_) => c_.copy(startTimeStamp = f_)) - def endTimeStamp: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.endTimeStamp)((c_, f_) => c_.copy(endTimeStamp = f_)) - def operatorInfo: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics]] = field(_.operatorInfo)((c_, f_) => c_.copy(operatorInfo = f_)) - def operatorWorkerMapping: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping]] = field(_.operatorWorkerMapping)((c_, f_) => c_.copy(operatorWorkerMapping = f_)) + implicit class ExecutionStatsStoreLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionStatsStore]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.ExecutionStatsStore](_l) { + def startTimestamp: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.startTimestamp)((c_, f_) => c_.copy(startTimestamp = f_)) + def endTimestamp: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.endTimestamp)((c_, f_) => c_.copy(endTimestamp = f_)) + def operatorInfo: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics]] = field(_.operatorInfo)((c_, f_) => c_.copy(operatorInfo = f_)) + def operatorWorkerMapping: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.OperatorWorkerMapping]] = field(_.operatorWorkerMapping)((c_, f_) => c_.copy(operatorWorkerMapping = f_)) } - final val STARTTIMESTAMP_FIELD_NUMBER = 1 - final val ENDTIMESTAMP_FIELD_NUMBER = 2 + final val START_TIMESTAMP_FIELD_NUMBER = 1 + final val END_TIMESTAMP_FIELD_NUMBER = 2 final val OPERATOR_INFO_FIELD_NUMBER = 3 final val OPERATOR_WORKER_MAPPING_FIELD_NUMBER = 4 @transient - private[workflowruntimestate] val _typemapper_operatorInfo: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics)]] + private[common] val _typemapper_operatorInfo: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.ExecutionStatsStore.OperatorInfoEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics)]] def of( - startTimeStamp: _root_.scala.Long, - endTimeStamp: _root_.scala.Long, - operatorInfo: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics], - operatorWorkerMapping: _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore( - startTimeStamp, - endTimeStamp, + startTimestamp: _root_.scala.Long, + endTimestamp: _root_.scala.Long, + operatorInfo: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.OperatorMetrics], + operatorWorkerMapping: _root_.scala.Seq[edu.uci.ics.amber.engine.common.OperatorWorkerMapping] + ): _root_.edu.uci.ics.amber.engine.common.ExecutionStatsStore = _root_.edu.uci.ics.amber.engine.common.ExecutionStatsStore( + startTimestamp, + endTimestamp, operatorInfo, operatorWorkerMapping ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/FatalErrorType.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/FatalErrorType.scala similarity index 78% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/FatalErrorType.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/FatalErrorType.scala index fe34ee6210e..70e576621bd 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/FatalErrorType.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/FatalErrorType.scala @@ -3,14 +3,14 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common sealed abstract class FatalErrorType(val value: _root_.scala.Int) extends _root_.scalapb.GeneratedEnum { type EnumType = FatalErrorType def isCompilationError: _root_.scala.Boolean = false def isExecutionFailure: _root_.scala.Boolean = false - def companion: _root_.scalapb.GeneratedEnumCompanion[FatalErrorType] = edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType - final def asRecognized: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.Recognized] = if (isUnrecognized) _root_.scala.None else _root_.scala.Some(this.asInstanceOf[edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.Recognized]) + def companion: _root_.scalapb.GeneratedEnumCompanion[FatalErrorType] = edu.uci.ics.amber.engine.common.FatalErrorType + final def asRecognized: _root_.scala.Option[edu.uci.ics.amber.engine.common.FatalErrorType.Recognized] = if (isUnrecognized) _root_.scala.None else _root_.scala.Some(this.asInstanceOf[edu.uci.ics.amber.engine.common.FatalErrorType.Recognized]) } object FatalErrorType extends _root_.scalapb.GeneratedEnumCompanion[FatalErrorType] { @@ -39,6 +39,6 @@ object FatalErrorType extends _root_.scalapb.GeneratedEnumCompanion[FatalErrorTy case 1 => EXECUTION_FAILURE case __other => Unrecognized(__other) } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.EnumDescriptor = WorkflowruntimestateProto.javaDescriptor.getEnumTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.EnumDescriptor = WorkflowruntimestateProto.scalaDescriptor.enums(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.EnumDescriptor = WorkflowRuntimeStateProto.javaDescriptor.getEnumTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.EnumDescriptor = WorkflowRuntimeStateProto.scalaDescriptor.enums(0) } \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/InputPort.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/InputPort.scala similarity index 72% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/InputPort.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/InputPort.scala index 6d5791b4999..0ca7fdade96 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/InputPort.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/InputPort.scala @@ -3,14 +3,14 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflow +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class InputPort( - id: edu.uci.ics.amber.engine.common.workflow.PortIdentity = edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance, + id: edu.uci.ics.amber.engine.common.PortIdentity = edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance, displayName: _root_.scala.Predef.String = "", allowMultiLinks: _root_.scala.Boolean = false, - dependencies: _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflow.PortIdentity] = _root_.scala.Seq.empty + dependencies: _root_.scala.Seq[edu.uci.ics.amber.engine.common.PortIdentity] = _root_.scala.Seq.empty ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[InputPort] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -80,18 +80,18 @@ final case class InputPort( __m.writeTo(_output__) }; } - def withId(__v: edu.uci.ics.amber.engine.common.workflow.PortIdentity): InputPort = copy(id = __v) + def withId(__v: edu.uci.ics.amber.engine.common.PortIdentity): InputPort = copy(id = __v) def withDisplayName(__v: _root_.scala.Predef.String): InputPort = copy(displayName = __v) def withAllowMultiLinks(__v: _root_.scala.Boolean): InputPort = copy(allowMultiLinks = __v) def clearDependencies = copy(dependencies = _root_.scala.Seq.empty) - def addDependencies(__vs: edu.uci.ics.amber.engine.common.workflow.PortIdentity *): InputPort = addAllDependencies(__vs) - def addAllDependencies(__vs: Iterable[edu.uci.ics.amber.engine.common.workflow.PortIdentity]): InputPort = copy(dependencies = dependencies ++ __vs) - def withDependencies(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflow.PortIdentity]): InputPort = copy(dependencies = __v) + def addDependencies(__vs: edu.uci.ics.amber.engine.common.PortIdentity *): InputPort = addAllDependencies(__vs) + def addAllDependencies(__vs: Iterable[edu.uci.ics.amber.engine.common.PortIdentity]): InputPort = copy(dependencies = dependencies ++ __vs) + def withDependencies(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.PortIdentity]): InputPort = copy(dependencies = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = id - if (__t != edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance) __t else null } case 2 => { val __t = displayName @@ -114,48 +114,48 @@ final case class InputPort( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflow.InputPort.type = edu.uci.ics.amber.engine.common.workflow.InputPort + def companion: edu.uci.ics.amber.engine.common.InputPort.type = edu.uci.ics.amber.engine.common.InputPort // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.InputPort]) } -object InputPort extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflow.InputPort] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflow.InputPort] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflow.InputPort = { - var __id: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflow.PortIdentity] = _root_.scala.None +object InputPort extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.InputPort] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.InputPort] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.InputPort = { + var __id: _root_.scala.Option[edu.uci.ics.amber.engine.common.PortIdentity] = _root_.scala.None var __displayName: _root_.scala.Predef.String = "" var __allowMultiLinks: _root_.scala.Boolean = false - val __dependencies: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.workflow.PortIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.workflow.PortIdentity] + val __dependencies: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.PortIdentity] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.PortIdentity] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __id = _root_.scala.Some(__id.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflow.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __id = _root_.scala.Some(__id.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => __displayName = _input__.readStringRequireUtf8() case 24 => __allowMultiLinks = _input__.readBool() case 34 => - __dependencies += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflow.PortIdentity](_input__) + __dependencies += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PortIdentity](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflow.InputPort( - id = __id.getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance), + edu.uci.ics.amber.engine.common.InputPort( + id = __id.getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance), displayName = __displayName, allowMultiLinks = __allowMultiLinks, dependencies = __dependencies.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflow.InputPort] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.InputPort] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflow.InputPort( - id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.workflow.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance), + edu.uci.ics.amber.engine.common.InputPort( + id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance), displayName = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), allowMultiLinks = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Boolean]).getOrElse(false), - dependencies = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.workflow.PortIdentity]]).getOrElse(_root_.scala.Seq.empty) + dependencies = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.PortIdentity]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } @@ -164,35 +164,35 @@ object InputPort extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.eng def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.workflow.PortIdentity - case 4 => __out = edu.uci.ics.amber.engine.common.workflow.PortIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.PortIdentity + case 4 => __out = edu.uci.ics.amber.engine.common.PortIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflow.InputPort( - id = edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance, + lazy val defaultInstance = edu.uci.ics.amber.engine.common.InputPort( + id = edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance, displayName = "", allowMultiLinks = false, dependencies = _root_.scala.Seq.empty ) - implicit class InputPortLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.InputPort]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflow.InputPort](_l) { - def id: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PortIdentity] = field(_.id)((c_, f_) => c_.copy(id = f_)) + implicit class InputPortLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.InputPort]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.InputPort](_l) { + def id: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PortIdentity] = field(_.id)((c_, f_) => c_.copy(id = f_)) def displayName: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.displayName)((c_, f_) => c_.copy(displayName = f_)) def allowMultiLinks: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Boolean] = field(_.allowMultiLinks)((c_, f_) => c_.copy(allowMultiLinks = f_)) - def dependencies: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflow.PortIdentity]] = field(_.dependencies)((c_, f_) => c_.copy(dependencies = f_)) + def dependencies: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.PortIdentity]] = field(_.dependencies)((c_, f_) => c_.copy(dependencies = f_)) } final val ID_FIELD_NUMBER = 1 - final val DISPLAYNAME_FIELD_NUMBER = 2 - final val ALLOWMULTILINKS_FIELD_NUMBER = 3 + final val DISPLAY_NAME_FIELD_NUMBER = 2 + final val ALLOW_MULTI_LINKS_FIELD_NUMBER = 3 final val DEPENDENCIES_FIELD_NUMBER = 4 def of( - id: edu.uci.ics.amber.engine.common.workflow.PortIdentity, + id: edu.uci.ics.amber.engine.common.PortIdentity, displayName: _root_.scala.Predef.String, allowMultiLinks: _root_.scala.Boolean, - dependencies: _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflow.PortIdentity] - ): _root_.edu.uci.ics.amber.engine.common.workflow.InputPort = _root_.edu.uci.ics.amber.engine.common.workflow.InputPort( + dependencies: _root_.scala.Seq[edu.uci.ics.amber.engine.common.PortIdentity] + ): _root_.edu.uci.ics.amber.engine.common.InputPort = _root_.edu.uci.ics.amber.engine.common.InputPort( id, displayName, allowMultiLinks, diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorBreakpoints.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorBreakpoints.scala similarity index 64% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorBreakpoints.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorBreakpoints.scala index a88ff9f2081..38a8b42250e 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorBreakpoints.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorBreakpoints.scala @@ -3,11 +3,11 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class OperatorBreakpoints( - unresolvedBreakpoints: _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault] = _root_.scala.Seq.empty + unresolvedBreakpoints: _root_.scala.Seq[edu.uci.ics.amber.engine.common.BreakpointFault] = _root_.scala.Seq.empty ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[OperatorBreakpoints] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -37,9 +37,9 @@ final case class OperatorBreakpoints( }; } def clearUnresolvedBreakpoints = copy(unresolvedBreakpoints = _root_.scala.Seq.empty) - def addUnresolvedBreakpoints(__vs: edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault *): OperatorBreakpoints = addAllUnresolvedBreakpoints(__vs) - def addAllUnresolvedBreakpoints(__vs: Iterable[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault]): OperatorBreakpoints = copy(unresolvedBreakpoints = unresolvedBreakpoints ++ __vs) - def withUnresolvedBreakpoints(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault]): OperatorBreakpoints = copy(unresolvedBreakpoints = __v) + def addUnresolvedBreakpoints(__vs: edu.uci.ics.amber.engine.common.BreakpointFault *): OperatorBreakpoints = addAllUnresolvedBreakpoints(__vs) + def addAllUnresolvedBreakpoints(__vs: Iterable[edu.uci.ics.amber.engine.common.BreakpointFault]): OperatorBreakpoints = copy(unresolvedBreakpoints = unresolvedBreakpoints ++ __vs) + def withUnresolvedBreakpoints(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.common.BreakpointFault]): OperatorBreakpoints = copy(unresolvedBreakpoints = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => unresolvedBreakpoints @@ -52,57 +52,57 @@ final case class OperatorBreakpoints( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints.type = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints + def companion: edu.uci.ics.amber.engine.common.OperatorBreakpoints.type = edu.uci.ics.amber.engine.common.OperatorBreakpoints // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.OperatorBreakpoints]) } -object OperatorBreakpoints extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints = { - val __unresolvedBreakpoints: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault] +object OperatorBreakpoints extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorBreakpoints] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorBreakpoints] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.OperatorBreakpoints = { + val __unresolvedBreakpoints: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.BreakpointFault] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.common.BreakpointFault] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __unresolvedBreakpoints += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault](_input__) + __unresolvedBreakpoints += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.BreakpointFault](_input__) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints( + edu.uci.ics.amber.engine.common.OperatorBreakpoints( unresolvedBreakpoints = __unresolvedBreakpoints.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.OperatorBreakpoints] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints( - unresolvedBreakpoints = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault]]).getOrElse(_root_.scala.Seq.empty) + edu.uci.ics.amber.engine.common.OperatorBreakpoints( + unresolvedBreakpoints = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.BreakpointFault]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowruntimestateProto.javaDescriptor.getMessageTypes().get(1) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowruntimestateProto.scalaDescriptor.messages(1) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowRuntimeStateProto.javaDescriptor.getMessageTypes().get(1) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowRuntimeStateProto.scalaDescriptor.messages(1) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault + case 1 => __out = edu.uci.ics.amber.engine.common.BreakpointFault } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.OperatorBreakpoints( unresolvedBreakpoints = _root_.scala.Seq.empty ) - implicit class OperatorBreakpointsLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints](_l) { - def unresolvedBreakpoints: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault]] = field(_.unresolvedBreakpoints)((c_, f_) => c_.copy(unresolvedBreakpoints = f_)) + implicit class OperatorBreakpointsLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OperatorBreakpoints]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.OperatorBreakpoints](_l) { + def unresolvedBreakpoints: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.common.BreakpointFault]] = field(_.unresolvedBreakpoints)((c_, f_) => c_.copy(unresolvedBreakpoints = f_)) } final val UNRESOLVED_BREAKPOINTS_FIELD_NUMBER = 1 def of( - unresolvedBreakpoints: _root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints( + unresolvedBreakpoints: _root_.scala.Seq[edu.uci.ics.amber.engine.common.BreakpointFault] + ): _root_.edu.uci.ics.amber.engine.common.OperatorBreakpoints = _root_.edu.uci.ics.amber.engine.common.OperatorBreakpoints( unresolvedBreakpoints ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorBreakpoints]) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorConsole.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorConsole.scala similarity index 58% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorConsole.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorConsole.scala index d81bed28a42..2229d6a0202 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorConsole.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorConsole.scala @@ -3,12 +3,12 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class OperatorConsole( - consoleMessages: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage] = _root_.scala.Seq.empty, - evaluateExprResults: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList] = _root_.scala.collection.immutable.Map.empty + consoleMessages: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage] = _root_.scala.Seq.empty, + evaluateExprResults: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList] = _root_.scala.collection.immutable.Map.empty ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[OperatorConsole] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -19,7 +19,7 @@ final case class OperatorConsole( __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } evaluateExprResults.foreach { __item => - val __value = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole._typemapper_evaluateExprResults.toBase(__item) + val __value = edu.uci.ics.amber.engine.common.OperatorConsole._typemapper_evaluateExprResults.toBase(__item) __size += 1 + _root_.com.google.protobuf.CodedOutputStream.computeUInt32SizeNoTag(__value.serializedSize) + __value.serializedSize } __size @@ -41,92 +41,92 @@ final case class OperatorConsole( __m.writeTo(_output__) }; evaluateExprResults.foreach { __v => - val __m = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole._typemapper_evaluateExprResults.toBase(__v) + val __m = edu.uci.ics.amber.engine.common.OperatorConsole._typemapper_evaluateExprResults.toBase(__v) _output__.writeTag(2, 2) _output__.writeUInt32NoTag(__m.serializedSize) __m.writeTo(_output__) }; } def clearConsoleMessages = copy(consoleMessages = _root_.scala.Seq.empty) - def addConsoleMessages(__vs: edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage *): OperatorConsole = addAllConsoleMessages(__vs) - def addAllConsoleMessages(__vs: Iterable[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage]): OperatorConsole = copy(consoleMessages = consoleMessages ++ __vs) - def withConsoleMessages(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage]): OperatorConsole = copy(consoleMessages = __v) + def addConsoleMessages(__vs: edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage *): OperatorConsole = addAllConsoleMessages(__vs) + def addAllConsoleMessages(__vs: Iterable[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage]): OperatorConsole = copy(consoleMessages = consoleMessages ++ __vs) + def withConsoleMessages(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage]): OperatorConsole = copy(consoleMessages = __v) def clearEvaluateExprResults = copy(evaluateExprResults = _root_.scala.collection.immutable.Map.empty) - def addEvaluateExprResults(__vs: (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList) *): OperatorConsole = addAllEvaluateExprResults(__vs) - def addAllEvaluateExprResults(__vs: Iterable[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList)]): OperatorConsole = copy(evaluateExprResults = evaluateExprResults ++ __vs) - def withEvaluateExprResults(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList]): OperatorConsole = copy(evaluateExprResults = __v) + def addEvaluateExprResults(__vs: (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList) *): OperatorConsole = addAllEvaluateExprResults(__vs) + def addAllEvaluateExprResults(__vs: Iterable[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList)]): OperatorConsole = copy(evaluateExprResults = evaluateExprResults ++ __vs) + def withEvaluateExprResults(__v: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList]): OperatorConsole = copy(evaluateExprResults = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => consoleMessages - case 2 => evaluateExprResults.iterator.map(edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole._typemapper_evaluateExprResults.toBase(_)).toSeq + case 2 => evaluateExprResults.iterator.map(edu.uci.ics.amber.engine.common.OperatorConsole._typemapper_evaluateExprResults.toBase(_)).toSeq } } def getField(__field: _root_.scalapb.descriptors.FieldDescriptor): _root_.scalapb.descriptors.PValue = { _root_.scala.Predef.require(__field.containingMessage eq companion.scalaDescriptor) (__field.number: @_root_.scala.unchecked) match { case 1 => _root_.scalapb.descriptors.PRepeated(consoleMessages.iterator.map(_.toPMessage).toVector) - case 2 => _root_.scalapb.descriptors.PRepeated(evaluateExprResults.iterator.map(edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole._typemapper_evaluateExprResults.toBase(_).toPMessage).toVector) + case 2 => _root_.scalapb.descriptors.PRepeated(evaluateExprResults.iterator.map(edu.uci.ics.amber.engine.common.OperatorConsole._typemapper_evaluateExprResults.toBase(_).toPMessage).toVector) } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.type = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole + def companion: edu.uci.ics.amber.engine.common.OperatorConsole.type = edu.uci.ics.amber.engine.common.OperatorConsole // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.OperatorConsole]) } -object OperatorConsole extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole = { - val __consoleMessages: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage] - val __evaluateExprResults: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList] +object OperatorConsole extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorConsole] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorConsole] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.OperatorConsole = { + val __consoleMessages: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage] + val __evaluateExprResults: _root_.scala.collection.mutable.Builder[(_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList), _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList]] = _root_.scala.collection.immutable.Map.newBuilder[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList] var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __consoleMessages += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage](_input__) + __consoleMessages += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage](_input__) case 18 => - __evaluateExprResults += edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole._typemapper_evaluateExprResults.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry](_input__)) + __evaluateExprResults += edu.uci.ics.amber.engine.common.OperatorConsole._typemapper_evaluateExprResults.toCustom(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry](_input__)) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole( + edu.uci.ics.amber.engine.common.OperatorConsole( consoleMessages = __consoleMessages.result(), evaluateExprResults = __evaluateExprResults.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.OperatorConsole] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole( - consoleMessages = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage]]).getOrElse(_root_.scala.Seq.empty), - evaluateExprResults = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole._typemapper_evaluateExprResults.toCustom(_)).toMap + edu.uci.ics.amber.engine.common.OperatorConsole( + consoleMessages = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage]]).getOrElse(_root_.scala.Seq.empty), + evaluateExprResults = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry]]).getOrElse(_root_.scala.Seq.empty).iterator.map(edu.uci.ics.amber.engine.common.OperatorConsole._typemapper_evaluateExprResults.toCustom(_)).toMap ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowruntimestateProto.javaDescriptor.getMessageTypes().get(4) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowruntimestateProto.scalaDescriptor.messages(4) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowRuntimeStateProto.javaDescriptor.getMessageTypes().get(4) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowRuntimeStateProto.scalaDescriptor.messages(4) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage - case 2 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry + case 1 => __out = edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage + case 2 => __out = edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry + _root_.edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry ) def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.OperatorConsole( consoleMessages = _root_.scala.Seq.empty, evaluateExprResults = _root_.scala.collection.immutable.Map.empty ) @SerialVersionUID(0L) final case class EvaluateExprResultsEntry( key: _root_.scala.Predef.String = "", - value: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList] = _root_.scala.None + value: _root_.scala.Option[edu.uci.ics.amber.engine.common.EvaluatedValueList] = _root_.scala.None ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[EvaluateExprResultsEntry] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -169,9 +169,9 @@ object OperatorConsole extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amb }; } def withKey(__v: _root_.scala.Predef.String): EvaluateExprResultsEntry = copy(key = __v) - def getValue: edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList = value.getOrElse(edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList.defaultInstance) + def getValue: edu.uci.ics.amber.engine.common.EvaluatedValueList = value.getOrElse(edu.uci.ics.amber.engine.common.EvaluatedValueList.defaultInstance) def clearValue: EvaluateExprResultsEntry = copy(value = _root_.scala.None) - def withValue(__v: edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList): EvaluateExprResultsEntry = copy(value = Option(__v)) + def withValue(__v: edu.uci.ics.amber.engine.common.EvaluatedValueList): EvaluateExprResultsEntry = copy(value = Option(__v)) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -189,15 +189,15 @@ object OperatorConsole extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amb } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry.type = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry + def companion: edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry.type = edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry]) } - object EvaluateExprResultsEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry = { + object EvaluateExprResultsEntry extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry = { var __key: _root_.scala.Predef.String = "" - var __value: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList] = _root_.scala.None + var __value: _root_.scala.Option[edu.uci.ics.amber.engine.common.EvaluatedValueList] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() @@ -206,71 +206,71 @@ object OperatorConsole extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amb case 10 => __key = _input__.readStringRequireUtf8() case 18 => - __value = Option(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __value = Option(__value.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.EvaluatedValueList](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry( + edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry( key = __key, value = __value ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry( + edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry( key = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), - value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList]]) + value = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).flatMap(_.as[_root_.scala.Option[edu.uci.ics.amber.engine.common.EvaluatedValueList]]) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.javaDescriptor.getNestedTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.scalaDescriptor.nestedMessages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = edu.uci.ics.amber.engine.common.OperatorConsole.javaDescriptor.getNestedTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = edu.uci.ics.amber.engine.common.OperatorConsole.scalaDescriptor.nestedMessages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList + case 2 => __out = edu.uci.ics.amber.engine.common.EvaluatedValueList } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry( key = "", value = _root_.scala.None ) - implicit class EvaluateExprResultsEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry](_l) { + implicit class EvaluateExprResultsEntryLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry](_l) { def key: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.key)((c_, f_) => c_.copy(key = f_)) - def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList] = field(_.getValue)((c_, f_) => c_.copy(value = Option(f_))) - def optionalValue: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList]] = field(_.value)((c_, f_) => c_.copy(value = f_)) + def value: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.EvaluatedValueList] = field(_.getValue)((c_, f_) => c_.copy(value = Option(f_))) + def optionalValue: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Option[edu.uci.ics.amber.engine.common.EvaluatedValueList]] = field(_.value)((c_, f_) => c_.copy(value = f_)) } final val KEY_FIELD_NUMBER = 1 final val VALUE_FIELD_NUMBER = 2 @transient - implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList)] = - _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList)](__m => (__m.key, __m.getValue))(__p => edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry(__p._1, Some(__p._2))) + implicit val keyValueMapper: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList)] = + _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList)](__m => (__m.key, __m.getValue))(__p => edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry(__p._1, Some(__p._2))) def of( key: _root_.scala.Predef.String, - value: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry( + value: _root_.scala.Option[edu.uci.ics.amber.engine.common.EvaluatedValueList] + ): _root_.edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry = _root_.edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry( key, value ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry]) } - implicit class OperatorConsoleLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole](_l) { - def consoleMessages: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage]] = field(_.consoleMessages)((c_, f_) => c_.copy(consoleMessages = f_)) - def evaluateExprResults: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList]] = field(_.evaluateExprResults)((c_, f_) => c_.copy(evaluateExprResults = f_)) + implicit class OperatorConsoleLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OperatorConsole]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.OperatorConsole](_l) { + def consoleMessages: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage]] = field(_.consoleMessages)((c_, f_) => c_.copy(consoleMessages = f_)) + def evaluateExprResults: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList]] = field(_.evaluateExprResults)((c_, f_) => c_.copy(evaluateExprResults = f_)) } final val CONSOLE_MESSAGES_FIELD_NUMBER = 1 final val EVALUATE_EXPR_RESULTS_FIELD_NUMBER = 2 @transient - private[workflowruntimestate] val _typemapper_evaluateExprResults: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole.EvaluateExprResultsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList)]] + private[common] val _typemapper_evaluateExprResults: _root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList)] = implicitly[_root_.scalapb.TypeMapper[edu.uci.ics.amber.engine.common.OperatorConsole.EvaluateExprResultsEntry, (_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList)]] def of( - consoleMessages: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ConsoleMessage], - evaluateExprResults: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole( + consoleMessages: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.rpc.ConsoleMessage], + evaluateExprResults: _root_.scala.collection.immutable.Map[_root_.scala.Predef.String, edu.uci.ics.amber.engine.common.EvaluatedValueList] + ): _root_.edu.uci.ics.amber.engine.common.OperatorConsole = _root_.edu.uci.ics.amber.engine.common.OperatorConsole( consoleMessages, evaluateExprResults ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/OperatorIdentity.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorIdentity.scala similarity index 76% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/OperatorIdentity.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorIdentity.scala index e553d797887..8cf5b2300e2 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/OperatorIdentity.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorIdentity.scala @@ -3,7 +3,7 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.virtualidentity +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class OperatorIdentity( @@ -55,13 +55,13 @@ final case class OperatorIdentity( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity.type = edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity + def companion: edu.uci.ics.amber.engine.common.OperatorIdentity.type = edu.uci.ics.amber.engine.common.OperatorIdentity // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.OperatorIdentity]) } -object OperatorIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity = { +object OperatorIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorIdentity] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorIdentity] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.OperatorIdentity = { var __id: _root_.scala.Predef.String = "" var _done__ = false while (!_done__) { @@ -73,33 +73,33 @@ object OperatorIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.am case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity( + edu.uci.ics.amber.engine.common.OperatorIdentity( id = __id ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.OperatorIdentity] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity( + edu.uci.ics.amber.engine.common.OperatorIdentity( id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualidentityProto.javaDescriptor.getMessageTypes().get(4) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualidentityProto.scalaDescriptor.messages(4) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualIdentityProto.javaDescriptor.getMessageTypes().get(4) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualIdentityProto.scalaDescriptor.messages(4) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.OperatorIdentity( id = "" ) - implicit class OperatorIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity](_l) { + implicit class OperatorIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OperatorIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.OperatorIdentity](_l) { def id: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.id)((c_, f_) => c_.copy(id = f_)) } final val ID_FIELD_NUMBER = 1 def of( id: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity = _root_.edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity( + ): _root_.edu.uci.ics.amber.engine.common.OperatorIdentity = _root_.edu.uci.ics.amber.engine.common.OperatorIdentity( id ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorIdentity]) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorMetrics.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorMetrics.scala similarity index 54% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorMetrics.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorMetrics.scala index 89da35fcead..31bac6d2d6f 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorMetrics.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorMetrics.scala @@ -3,12 +3,12 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class OperatorMetrics( - operatorState: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.UNINITIALIZED, - operatorStatistics: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics.defaultInstance + operatorState: edu.uci.ics.amber.engine.common.WorkflowAggregatedState = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.UNINITIALIZED, + operatorStatistics: edu.uci.ics.amber.engine.common.OperatorStatistics = edu.uci.ics.amber.engine.common.OperatorStatistics.defaultInstance ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[OperatorMetrics] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -55,8 +55,8 @@ final case class OperatorMetrics( } }; } - def withOperatorState(__v: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState): OperatorMetrics = copy(operatorState = __v) - def withOperatorStatistics(__v: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics): OperatorMetrics = copy(operatorStatistics = __v) + def withOperatorState(__v: edu.uci.ics.amber.engine.common.WorkflowAggregatedState): OperatorMetrics = copy(operatorState = __v) + def withOperatorStatistics(__v: edu.uci.ics.amber.engine.common.OperatorStatistics): OperatorMetrics = copy(operatorStatistics = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { @@ -65,7 +65,7 @@ final case class OperatorMetrics( } case 2 => { val __t = operatorStatistics - if (__t != edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.OperatorStatistics.defaultInstance) __t else null } } } @@ -77,70 +77,70 @@ final case class OperatorMetrics( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics.type = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics + def companion: edu.uci.ics.amber.engine.common.OperatorMetrics.type = edu.uci.ics.amber.engine.common.OperatorMetrics // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.OperatorMetrics]) } -object OperatorMetrics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics = { - var __operatorState: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.UNINITIALIZED - var __operatorStatistics: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics] = _root_.scala.None +object OperatorMetrics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorMetrics] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorMetrics] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.OperatorMetrics = { + var __operatorState: edu.uci.ics.amber.engine.common.WorkflowAggregatedState = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.UNINITIALIZED + var __operatorStatistics: _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorStatistics] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 8 => - __operatorState = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.fromValue(_input__.readEnum()) + __operatorState = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.fromValue(_input__.readEnum()) case 18 => - __operatorStatistics = _root_.scala.Some(__operatorStatistics.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __operatorStatistics = _root_.scala.Some(__operatorStatistics.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.OperatorStatistics](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics( + edu.uci.ics.amber.engine.common.OperatorMetrics( operatorState = __operatorState, - operatorStatistics = __operatorStatistics.getOrElse(edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics.defaultInstance) + operatorStatistics = __operatorStatistics.getOrElse(edu.uci.ics.amber.engine.common.OperatorStatistics.defaultInstance) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.OperatorMetrics] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics( - operatorState = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.UNINITIALIZED.scalaValueDescriptor).number), - operatorStatistics = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics]).getOrElse(edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics.defaultInstance) + edu.uci.ics.amber.engine.common.OperatorMetrics( + operatorState = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.common.WorkflowAggregatedState.UNINITIALIZED.scalaValueDescriptor).number), + operatorStatistics = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.OperatorStatistics]).getOrElse(edu.uci.ics.amber.engine.common.OperatorStatistics.defaultInstance) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowruntimestateProto.javaDescriptor.getMessageTypes().get(8) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowruntimestateProto.scalaDescriptor.messages(8) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowMetricsProto.javaDescriptor.getMessageTypes().get(1) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowMetricsProto.scalaDescriptor.messages(1) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 2 => __out = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics + case 2 => __out = edu.uci.ics.amber.engine.common.OperatorStatistics } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = { (__fieldNumber: @_root_.scala.unchecked) match { - case 1 => edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState + case 1 => edu.uci.ics.amber.engine.common.WorkflowAggregatedState } } - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics( - operatorState = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.UNINITIALIZED, - operatorStatistics = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics.defaultInstance + lazy val defaultInstance = edu.uci.ics.amber.engine.common.OperatorMetrics( + operatorState = edu.uci.ics.amber.engine.common.WorkflowAggregatedState.UNINITIALIZED, + operatorStatistics = edu.uci.ics.amber.engine.common.OperatorStatistics.defaultInstance ) - implicit class OperatorMetricsLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics](_l) { - def operatorState: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState] = field(_.operatorState)((c_, f_) => c_.copy(operatorState = f_)) - def operatorStatistics: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics] = field(_.operatorStatistics)((c_, f_) => c_.copy(operatorStatistics = f_)) + implicit class OperatorMetricsLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OperatorMetrics]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.OperatorMetrics](_l) { + def operatorState: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.WorkflowAggregatedState] = field(_.operatorState)((c_, f_) => c_.copy(operatorState = f_)) + def operatorStatistics: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OperatorStatistics] = field(_.operatorStatistics)((c_, f_) => c_.copy(operatorStatistics = f_)) } final val OPERATOR_STATE_FIELD_NUMBER = 1 final val OPERATOR_STATISTICS_FIELD_NUMBER = 2 def of( - operatorState: edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState, - operatorStatistics: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics( + operatorState: edu.uci.ics.amber.engine.common.WorkflowAggregatedState, + operatorStatistics: edu.uci.ics.amber.engine.common.OperatorStatistics + ): _root_.edu.uci.ics.amber.engine.common.OperatorMetrics = _root_.edu.uci.ics.amber.engine.common.OperatorMetrics( operatorState, operatorStatistics ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorStatistics.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorStatistics.scala similarity index 75% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorStatistics.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorStatistics.scala index fcaa8bde7f3..101a1babcff 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorStatistics.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorStatistics.scala @@ -3,12 +3,12 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class OperatorStatistics( - inputCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] = _root_.scala.Seq.empty, - outputCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] = _root_.scala.Seq.empty, + inputCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] = _root_.scala.Seq.empty, + outputCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] = _root_.scala.Seq.empty, numWorkers: _root_.scala.Int = 0, dataProcessingTime: _root_.scala.Long = 0L, controlProcessingTime: _root_.scala.Long = 0L, @@ -104,13 +104,13 @@ final case class OperatorStatistics( }; } def clearInputCount = copy(inputCount = _root_.scala.Seq.empty) - def addInputCount(__vs: edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping *): OperatorStatistics = addAllInputCount(__vs) - def addAllInputCount(__vs: Iterable[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]): OperatorStatistics = copy(inputCount = inputCount ++ __vs) - def withInputCount(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]): OperatorStatistics = copy(inputCount = __v) + def addInputCount(__vs: edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping *): OperatorStatistics = addAllInputCount(__vs) + def addAllInputCount(__vs: Iterable[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]): OperatorStatistics = copy(inputCount = inputCount ++ __vs) + def withInputCount(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]): OperatorStatistics = copy(inputCount = __v) def clearOutputCount = copy(outputCount = _root_.scala.Seq.empty) - def addOutputCount(__vs: edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping *): OperatorStatistics = addAllOutputCount(__vs) - def addAllOutputCount(__vs: Iterable[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]): OperatorStatistics = copy(outputCount = outputCount ++ __vs) - def withOutputCount(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]): OperatorStatistics = copy(outputCount = __v) + def addOutputCount(__vs: edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping *): OperatorStatistics = addAllOutputCount(__vs) + def addAllOutputCount(__vs: Iterable[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]): OperatorStatistics = copy(outputCount = outputCount ++ __vs) + def withOutputCount(__v: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]): OperatorStatistics = copy(outputCount = __v) def withNumWorkers(__v: _root_.scala.Int): OperatorStatistics = copy(numWorkers = __v) def withDataProcessingTime(__v: _root_.scala.Long): OperatorStatistics = copy(dataProcessingTime = __v) def withControlProcessingTime(__v: _root_.scala.Long): OperatorStatistics = copy(controlProcessingTime = __v) @@ -149,15 +149,15 @@ final case class OperatorStatistics( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics.type = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics + def companion: edu.uci.ics.amber.engine.common.OperatorStatistics.type = edu.uci.ics.amber.engine.common.OperatorStatistics // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.OperatorStatistics]) } -object OperatorStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics = { - val __inputCount: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] - val __outputCount: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping] +object OperatorStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorStatistics] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorStatistics] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.OperatorStatistics = { + val __inputCount: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] + val __outputCount: _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] = new _root_.scala.collection.immutable.VectorBuilder[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping] var __numWorkers: _root_.scala.Int = 0 var __dataProcessingTime: _root_.scala.Long = 0L var __controlProcessingTime: _root_.scala.Long = 0L @@ -168,9 +168,9 @@ object OperatorStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics. _tag__ match { case 0 => _done__ = true case 10 => - __inputCount += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping](_input__) + __inputCount += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping](_input__) case 18 => - __outputCount += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping](_input__) + __outputCount += _root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping](_input__) case 24 => __numWorkers = _input__.readInt32() case 32 => @@ -182,7 +182,7 @@ object OperatorStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics. case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics( + edu.uci.ics.amber.engine.common.OperatorStatistics( inputCount = __inputCount.result(), outputCount = __outputCount.result(), numWorkers = __numWorkers, @@ -191,12 +191,12 @@ object OperatorStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics. idleTime = __idleTime ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.OperatorStatistics] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics( - inputCount = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]]).getOrElse(_root_.scala.Seq.empty), - outputCount = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]]).getOrElse(_root_.scala.Seq.empty), + edu.uci.ics.amber.engine.common.OperatorStatistics( + inputCount = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]]).getOrElse(_root_.scala.Seq.empty), + outputCount = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]]).getOrElse(_root_.scala.Seq.empty), numWorkers = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Int]).getOrElse(0), dataProcessingTime = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Long]).getOrElse(0L), controlProcessingTime = __fieldsMap.get(scalaDescriptor.findFieldByNumber(5).get).map(_.as[_root_.scala.Long]).getOrElse(0L), @@ -204,19 +204,19 @@ object OperatorStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics. ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowruntimestateProto.javaDescriptor.getMessageTypes().get(7) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowruntimestateProto.scalaDescriptor.messages(7) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowMetricsProto.javaDescriptor.getMessageTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowMetricsProto.scalaDescriptor.messages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping - case 2 => __out = edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping + case 1 => __out = edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping + case 2 => __out = edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.OperatorStatistics( inputCount = _root_.scala.Seq.empty, outputCount = _root_.scala.Seq.empty, numWorkers = 0, @@ -224,9 +224,9 @@ object OperatorStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics. controlProcessingTime = 0L, idleTime = 0L ) - implicit class OperatorStatisticsLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics](_l) { - def inputCount: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]] = field(_.inputCount)((c_, f_) => c_.copy(inputCount = f_)) - def outputCount: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping]] = field(_.outputCount)((c_, f_) => c_.copy(outputCount = f_)) + implicit class OperatorStatisticsLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OperatorStatistics]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.OperatorStatistics](_l) { + def inputCount: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]] = field(_.inputCount)((c_, f_) => c_.copy(inputCount = f_)) + def outputCount: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping]] = field(_.outputCount)((c_, f_) => c_.copy(outputCount = f_)) def numWorkers: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.numWorkers)((c_, f_) => c_.copy(numWorkers = f_)) def dataProcessingTime: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.dataProcessingTime)((c_, f_) => c_.copy(dataProcessingTime = f_)) def controlProcessingTime: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.controlProcessingTime)((c_, f_) => c_.copy(controlProcessingTime = f_)) @@ -239,13 +239,13 @@ object OperatorStatistics extends scalapb.GeneratedMessageCompanion[edu.uci.ics. final val CONTROL_PROCESSING_TIME_FIELD_NUMBER = 5 final val IDLE_TIME_FIELD_NUMBER = 6 def of( - inputCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping], - outputCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.statistics.PortTupleCountMapping], + inputCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping], + outputCount: _root_.scala.Seq[edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping], numWorkers: _root_.scala.Int, dataProcessingTime: _root_.scala.Long, controlProcessingTime: _root_.scala.Long, idleTime: _root_.scala.Long - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics( + ): _root_.edu.uci.ics.amber.engine.common.OperatorStatistics = _root_.edu.uci.ics.amber.engine.common.OperatorStatistics( inputCount, outputCount, numWorkers, diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorWorkerMapping.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorWorkerMapping.scala similarity index 79% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorWorkerMapping.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorWorkerMapping.scala index 346ba1263ca..4c2ffd098c0 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/OperatorWorkerMapping.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OperatorWorkerMapping.scala @@ -3,7 +3,7 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class OperatorWorkerMapping( @@ -70,13 +70,13 @@ final case class OperatorWorkerMapping( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping.type = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping + def companion: edu.uci.ics.amber.engine.common.OperatorWorkerMapping.type = edu.uci.ics.amber.engine.common.OperatorWorkerMapping // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.OperatorWorkerMapping]) } -object OperatorWorkerMapping extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping = { +object OperatorWorkerMapping extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorWorkerMapping] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OperatorWorkerMapping] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.OperatorWorkerMapping = { var __operatorId: _root_.scala.Predef.String = "" val __workerIds: _root_.scala.collection.immutable.VectorBuilder[_root_.scala.Predef.String] = new _root_.scala.collection.immutable.VectorBuilder[_root_.scala.Predef.String] var _done__ = false @@ -91,39 +91,39 @@ object OperatorWorkerMapping extends scalapb.GeneratedMessageCompanion[edu.uci.i case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping( + edu.uci.ics.amber.engine.common.OperatorWorkerMapping( operatorId = __operatorId, workerIds = __workerIds.result() ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.OperatorWorkerMapping] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping( + edu.uci.ics.amber.engine.common.OperatorWorkerMapping( operatorId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), workerIds = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Seq[_root_.scala.Predef.String]]).getOrElse(_root_.scala.Seq.empty) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowruntimestateProto.javaDescriptor.getMessageTypes().get(6) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowruntimestateProto.scalaDescriptor.messages(6) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowRuntimeStateProto.javaDescriptor.getMessageTypes().get(6) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowRuntimeStateProto.scalaDescriptor.messages(6) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.OperatorWorkerMapping( operatorId = "", workerIds = _root_.scala.Seq.empty ) - implicit class OperatorWorkerMappingLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping](_l) { + implicit class OperatorWorkerMappingLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OperatorWorkerMapping]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.OperatorWorkerMapping](_l) { def operatorId: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.operatorId)((c_, f_) => c_.copy(operatorId = f_)) def workerIds: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Seq[_root_.scala.Predef.String]] = field(_.workerIds)((c_, f_) => c_.copy(workerIds = f_)) } - final val OPERATORID_FIELD_NUMBER = 1 - final val WORKERIDS_FIELD_NUMBER = 2 + final val OPERATOR_ID_FIELD_NUMBER = 1 + final val WORKER_IDS_FIELD_NUMBER = 2 def of( operatorId: _root_.scala.Predef.String, workerIds: _root_.scala.Seq[_root_.scala.Predef.String] - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping( + ): _root_.edu.uci.ics.amber.engine.common.OperatorWorkerMapping = _root_.edu.uci.ics.amber.engine.common.OperatorWorkerMapping( operatorId, workerIds ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/OutputPort.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OutputPort.scala similarity index 75% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/OutputPort.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OutputPort.scala index 7823ecb0af6..e407073b167 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/OutputPort.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/OutputPort.scala @@ -3,11 +3,11 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflow +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class OutputPort( - id: edu.uci.ics.amber.engine.common.workflow.PortIdentity = edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance, + id: edu.uci.ics.amber.engine.common.PortIdentity = edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance, displayName: _root_.scala.Predef.String = "", blocking: _root_.scala.Boolean = false ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[OutputPort] { @@ -69,14 +69,14 @@ final case class OutputPort( } }; } - def withId(__v: edu.uci.ics.amber.engine.common.workflow.PortIdentity): OutputPort = copy(id = __v) + def withId(__v: edu.uci.ics.amber.engine.common.PortIdentity): OutputPort = copy(id = __v) def withDisplayName(__v: _root_.scala.Predef.String): OutputPort = copy(displayName = __v) def withBlocking(__v: _root_.scala.Boolean): OutputPort = copy(blocking = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = id - if (__t != edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance) __t else null } case 2 => { val __t = displayName @@ -97,14 +97,14 @@ final case class OutputPort( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflow.OutputPort.type = edu.uci.ics.amber.engine.common.workflow.OutputPort + def companion: edu.uci.ics.amber.engine.common.OutputPort.type = edu.uci.ics.amber.engine.common.OutputPort // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.OutputPort]) } -object OutputPort extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflow.OutputPort] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflow.OutputPort] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflow.OutputPort = { - var __id: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflow.PortIdentity] = _root_.scala.None +object OutputPort extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OutputPort] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.OutputPort] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.OutputPort = { + var __id: _root_.scala.Option[edu.uci.ics.amber.engine.common.PortIdentity] = _root_.scala.None var __displayName: _root_.scala.Predef.String = "" var __blocking: _root_.scala.Boolean = false var _done__ = false @@ -113,7 +113,7 @@ object OutputPort extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.en _tag__ match { case 0 => _done__ = true case 10 => - __id = _root_.scala.Some(__id.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflow.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __id = _root_.scala.Some(__id.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => __displayName = _input__.readStringRequireUtf8() case 24 => @@ -121,17 +121,17 @@ object OutputPort extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.en case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflow.OutputPort( - id = __id.getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance), + edu.uci.ics.amber.engine.common.OutputPort( + id = __id.getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance), displayName = __displayName, blocking = __blocking ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflow.OutputPort] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.OutputPort] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflow.OutputPort( - id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.workflow.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance), + edu.uci.ics.amber.engine.common.OutputPort( + id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance), displayName = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), blocking = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Boolean]).getOrElse(false) ) @@ -142,30 +142,30 @@ object OutputPort extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.en def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.workflow.PortIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.PortIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflow.OutputPort( - id = edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance, + lazy val defaultInstance = edu.uci.ics.amber.engine.common.OutputPort( + id = edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance, displayName = "", blocking = false ) - implicit class OutputPortLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.OutputPort]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflow.OutputPort](_l) { - def id: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PortIdentity] = field(_.id)((c_, f_) => c_.copy(id = f_)) + implicit class OutputPortLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OutputPort]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.OutputPort](_l) { + def id: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PortIdentity] = field(_.id)((c_, f_) => c_.copy(id = f_)) def displayName: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.displayName)((c_, f_) => c_.copy(displayName = f_)) def blocking: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Boolean] = field(_.blocking)((c_, f_) => c_.copy(blocking = f_)) } final val ID_FIELD_NUMBER = 1 - final val DISPLAYNAME_FIELD_NUMBER = 2 + final val DISPLAY_NAME_FIELD_NUMBER = 2 final val BLOCKING_FIELD_NUMBER = 3 def of( - id: edu.uci.ics.amber.engine.common.workflow.PortIdentity, + id: edu.uci.ics.amber.engine.common.PortIdentity, displayName: _root_.scala.Predef.String, blocking: _root_.scala.Boolean - ): _root_.edu.uci.ics.amber.engine.common.workflow.OutputPort = _root_.edu.uci.ics.amber.engine.common.workflow.OutputPort( + ): _root_.edu.uci.ics.amber.engine.common.OutputPort = _root_.edu.uci.ics.amber.engine.common.OutputPort( id, displayName, blocking diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/PhysicalLink.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/PhysicalLink.scala similarity index 54% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/PhysicalLink.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/PhysicalLink.scala index 651ef332562..bb089ed350e 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/PhysicalLink.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/PhysicalLink.scala @@ -3,14 +3,14 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflow +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class PhysicalLink( - fromOpId: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance, - fromPortId: edu.uci.ics.amber.engine.common.workflow.PortIdentity = edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance, - toOpId: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance, - toPortId: edu.uci.ics.amber.engine.common.workflow.PortIdentity = edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance + fromOpId: edu.uci.ics.amber.engine.common.PhysicalOpIdentity = edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance, + fromPortId: edu.uci.ics.amber.engine.common.PortIdentity = edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance, + toOpId: edu.uci.ics.amber.engine.common.PhysicalOpIdentity = edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance, + toPortId: edu.uci.ics.amber.engine.common.PortIdentity = edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[PhysicalLink] { @transient private[this] var __serializedSizeMemoized: _root_.scala.Int = 0 @@ -89,27 +89,27 @@ final case class PhysicalLink( } }; } - def withFromOpId(__v: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity): PhysicalLink = copy(fromOpId = __v) - def withFromPortId(__v: edu.uci.ics.amber.engine.common.workflow.PortIdentity): PhysicalLink = copy(fromPortId = __v) - def withToOpId(__v: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity): PhysicalLink = copy(toOpId = __v) - def withToPortId(__v: edu.uci.ics.amber.engine.common.workflow.PortIdentity): PhysicalLink = copy(toPortId = __v) + def withFromOpId(__v: edu.uci.ics.amber.engine.common.PhysicalOpIdentity): PhysicalLink = copy(fromOpId = __v) + def withFromPortId(__v: edu.uci.ics.amber.engine.common.PortIdentity): PhysicalLink = copy(fromPortId = __v) + def withToOpId(__v: edu.uci.ics.amber.engine.common.PhysicalOpIdentity): PhysicalLink = copy(toOpId = __v) + def withToPortId(__v: edu.uci.ics.amber.engine.common.PortIdentity): PhysicalLink = copy(toPortId = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = fromOpId - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance) __t else null } case 2 => { val __t = fromPortId - if (__t != edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance) __t else null } case 3 => { val __t = toOpId - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance) __t else null } case 4 => { val __t = toPortId - if (__t != edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance) __t else null } } } @@ -123,48 +123,48 @@ final case class PhysicalLink( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflow.PhysicalLink.type = edu.uci.ics.amber.engine.common.workflow.PhysicalLink + def companion: edu.uci.ics.amber.engine.common.PhysicalLink.type = edu.uci.ics.amber.engine.common.PhysicalLink // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.PhysicalLink]) } -object PhysicalLink extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflow.PhysicalLink] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflow.PhysicalLink] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflow.PhysicalLink = { - var __fromOpId: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] = _root_.scala.None - var __fromPortId: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflow.PortIdentity] = _root_.scala.None - var __toOpId: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] = _root_.scala.None - var __toPortId: _root_.scala.Option[edu.uci.ics.amber.engine.common.workflow.PortIdentity] = _root_.scala.None +object PhysicalLink extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.PhysicalLink] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.PhysicalLink] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.PhysicalLink = { + var __fromOpId: _root_.scala.Option[edu.uci.ics.amber.engine.common.PhysicalOpIdentity] = _root_.scala.None + var __fromPortId: _root_.scala.Option[edu.uci.ics.amber.engine.common.PortIdentity] = _root_.scala.None + var __toOpId: _root_.scala.Option[edu.uci.ics.amber.engine.common.PhysicalOpIdentity] = _root_.scala.None + var __toPortId: _root_.scala.Option[edu.uci.ics.amber.engine.common.PortIdentity] = _root_.scala.None var _done__ = false while (!_done__) { val _tag__ = _input__.readTag() _tag__ match { case 0 => _done__ = true case 10 => - __fromOpId = _root_.scala.Some(__fromOpId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __fromOpId = _root_.scala.Some(__fromOpId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PhysicalOpIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => - __fromPortId = _root_.scala.Some(__fromPortId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflow.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __fromPortId = _root_.scala.Some(__fromPortId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 26 => - __toOpId = _root_.scala.Some(__toOpId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __toOpId = _root_.scala.Some(__toOpId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PhysicalOpIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 34 => - __toPortId = _root_.scala.Some(__toPortId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.workflow.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __toPortId = _root_.scala.Some(__toPortId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.PortIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflow.PhysicalLink( - fromOpId = __fromOpId.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance), - fromPortId = __fromPortId.getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance), - toOpId = __toOpId.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance), - toPortId = __toPortId.getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance) + edu.uci.ics.amber.engine.common.PhysicalLink( + fromOpId = __fromOpId.getOrElse(edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance), + fromPortId = __fromPortId.getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance), + toOpId = __toOpId.getOrElse(edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance), + toPortId = __toPortId.getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance) ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflow.PhysicalLink] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.PhysicalLink] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflow.PhysicalLink( - fromOpId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance), - fromPortId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.workflow.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance), - toOpId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance), - toPortId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[edu.uci.ics.amber.engine.common.workflow.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance) + edu.uci.ics.amber.engine.common.PhysicalLink( + fromOpId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]).getOrElse(edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance), + fromPortId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[edu.uci.ics.amber.engine.common.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance), + toOpId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]).getOrElse(edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance), + toPortId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[edu.uci.ics.amber.engine.common.PortIdentity]).getOrElse(edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance) ) case _ => throw new RuntimeException("Expected PMessage") } @@ -173,37 +173,37 @@ object PhysicalLink extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber. def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity - case 2 => __out = edu.uci.ics.amber.engine.common.workflow.PortIdentity - case 3 => __out = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity - case 4 => __out = edu.uci.ics.amber.engine.common.workflow.PortIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.PhysicalOpIdentity + case 2 => __out = edu.uci.ics.amber.engine.common.PortIdentity + case 3 => __out = edu.uci.ics.amber.engine.common.PhysicalOpIdentity + case 4 => __out = edu.uci.ics.amber.engine.common.PortIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflow.PhysicalLink( - fromOpId = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance, - fromPortId = edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance, - toOpId = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.defaultInstance, - toPortId = edu.uci.ics.amber.engine.common.workflow.PortIdentity.defaultInstance + lazy val defaultInstance = edu.uci.ics.amber.engine.common.PhysicalLink( + fromOpId = edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance, + fromPortId = edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance, + toOpId = edu.uci.ics.amber.engine.common.PhysicalOpIdentity.defaultInstance, + toPortId = edu.uci.ics.amber.engine.common.PortIdentity.defaultInstance ) - implicit class PhysicalLinkLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PhysicalLink]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PhysicalLink](_l) { - def fromOpId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] = field(_.fromOpId)((c_, f_) => c_.copy(fromOpId = f_)) - def fromPortId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PortIdentity] = field(_.fromPortId)((c_, f_) => c_.copy(fromPortId = f_)) - def toOpId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] = field(_.toOpId)((c_, f_) => c_.copy(toOpId = f_)) - def toPortId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PortIdentity] = field(_.toPortId)((c_, f_) => c_.copy(toPortId = f_)) + implicit class PhysicalLinkLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PhysicalLink]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.PhysicalLink](_l) { + def fromOpId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PhysicalOpIdentity] = field(_.fromOpId)((c_, f_) => c_.copy(fromOpId = f_)) + def fromPortId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PortIdentity] = field(_.fromPortId)((c_, f_) => c_.copy(fromPortId = f_)) + def toOpId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PhysicalOpIdentity] = field(_.toOpId)((c_, f_) => c_.copy(toOpId = f_)) + def toPortId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PortIdentity] = field(_.toPortId)((c_, f_) => c_.copy(toPortId = f_)) } - final val FROMOPID_FIELD_NUMBER = 1 - final val FROMPORTID_FIELD_NUMBER = 2 - final val TOOPID_FIELD_NUMBER = 3 - final val TOPORTID_FIELD_NUMBER = 4 + final val FROM_OP_ID_FIELD_NUMBER = 1 + final val FROM_PORT_ID_FIELD_NUMBER = 2 + final val TO_OP_ID_FIELD_NUMBER = 3 + final val TO_PORT_ID_FIELD_NUMBER = 4 def of( - fromOpId: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity, - fromPortId: edu.uci.ics.amber.engine.common.workflow.PortIdentity, - toOpId: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity, - toPortId: edu.uci.ics.amber.engine.common.workflow.PortIdentity - ): _root_.edu.uci.ics.amber.engine.common.workflow.PhysicalLink = _root_.edu.uci.ics.amber.engine.common.workflow.PhysicalLink( + fromOpId: edu.uci.ics.amber.engine.common.PhysicalOpIdentity, + fromPortId: edu.uci.ics.amber.engine.common.PortIdentity, + toOpId: edu.uci.ics.amber.engine.common.PhysicalOpIdentity, + toPortId: edu.uci.ics.amber.engine.common.PortIdentity + ): _root_.edu.uci.ics.amber.engine.common.PhysicalLink = _root_.edu.uci.ics.amber.engine.common.PhysicalLink( fromOpId, fromPortId, toOpId, diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/PhysicalOpIdentity.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/PhysicalOpIdentity.scala similarity index 67% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/PhysicalOpIdentity.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/PhysicalOpIdentity.scala index 3cf496f210b..702cfa95aa4 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/PhysicalOpIdentity.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/PhysicalOpIdentity.scala @@ -3,11 +3,11 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.virtualidentity +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class PhysicalOpIdentity( - logicalOpId: edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity, + logicalOpId: edu.uci.ics.amber.engine.common.OperatorIdentity, layerName: _root_.scala.Predef.String ) extends scalapb.GeneratedMessage with scalapb.lenses.Updatable[PhysicalOpIdentity] { @transient @@ -55,13 +55,13 @@ final case class PhysicalOpIdentity( } }; } - def withLogicalOpId(__v: edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity): PhysicalOpIdentity = copy(logicalOpId = __v) + def withLogicalOpId(__v: edu.uci.ics.amber.engine.common.OperatorIdentity): PhysicalOpIdentity = copy(logicalOpId = __v) def withLayerName(__v: _root_.scala.Predef.String): PhysicalOpIdentity = copy(layerName = __v) def getFieldByNumber(__fieldNumber: _root_.scala.Int): _root_.scala.Any = { (__fieldNumber: @_root_.scala.unchecked) match { case 1 => { val __t = logicalOpId - if (__t != edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity.defaultInstance) __t else null + if (__t != edu.uci.ics.amber.engine.common.OperatorIdentity.defaultInstance) __t else null } case 2 => { val __t = layerName @@ -77,14 +77,14 @@ final case class PhysicalOpIdentity( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity.type = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity + def companion: edu.uci.ics.amber.engine.common.PhysicalOpIdentity.type = edu.uci.ics.amber.engine.common.PhysicalOpIdentity // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.PhysicalOpIdentity]) } -object PhysicalOpIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity = { - var __logicalOpId: _root_.scala.Option[edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity] = _root_.scala.None +object PhysicalOpIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.PhysicalOpIdentity] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.PhysicalOpIdentity] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.PhysicalOpIdentity = { + var __logicalOpId: _root_.scala.Option[edu.uci.ics.amber.engine.common.OperatorIdentity] = _root_.scala.None var __layerName: _root_.scala.Predef.String = "" var _done__ = false while (!_done__) { @@ -92,51 +92,51 @@ object PhysicalOpIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics. _tag__ match { case 0 => _done__ = true case 10 => - __logicalOpId = _root_.scala.Some(__logicalOpId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) + __logicalOpId = _root_.scala.Some(__logicalOpId.fold(_root_.scalapb.LiteParser.readMessage[edu.uci.ics.amber.engine.common.OperatorIdentity](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 18 => __layerName = _input__.readStringRequireUtf8() case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity( - logicalOpId = __logicalOpId.getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity.defaultInstance), + edu.uci.ics.amber.engine.common.PhysicalOpIdentity( + logicalOpId = __logicalOpId.getOrElse(edu.uci.ics.amber.engine.common.OperatorIdentity.defaultInstance), layerName = __layerName ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.PhysicalOpIdentity] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity( - logicalOpId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity]).getOrElse(edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity.defaultInstance), + edu.uci.ics.amber.engine.common.PhysicalOpIdentity( + logicalOpId = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[edu.uci.ics.amber.engine.common.OperatorIdentity]).getOrElse(edu.uci.ics.amber.engine.common.OperatorIdentity.defaultInstance), layerName = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Predef.String]).getOrElse("") ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualidentityProto.javaDescriptor.getMessageTypes().get(5) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualidentityProto.scalaDescriptor.messages(5) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualIdentityProto.javaDescriptor.getMessageTypes().get(5) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualIdentityProto.scalaDescriptor.messages(5) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { - case 1 => __out = edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity + case 1 => __out = edu.uci.ics.amber.engine.common.OperatorIdentity } __out } lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity( - logicalOpId = edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity.defaultInstance, + lazy val defaultInstance = edu.uci.ics.amber.engine.common.PhysicalOpIdentity( + logicalOpId = edu.uci.ics.amber.engine.common.OperatorIdentity.defaultInstance, layerName = "" ) - implicit class PhysicalOpIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity](_l) { - def logicalOpId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity] = field(_.logicalOpId)((c_, f_) => c_.copy(logicalOpId = f_)) + implicit class PhysicalOpIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PhysicalOpIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.PhysicalOpIdentity](_l) { + def logicalOpId: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.OperatorIdentity] = field(_.logicalOpId)((c_, f_) => c_.copy(logicalOpId = f_)) def layerName: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.layerName)((c_, f_) => c_.copy(layerName = f_)) } - final val LOGICALOPID_FIELD_NUMBER = 1 - final val LAYERNAME_FIELD_NUMBER = 2 + final val LOGICAL_OP_ID_FIELD_NUMBER = 1 + final val LAYER_NAME_FIELD_NUMBER = 2 def of( - logicalOpId: edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity, + logicalOpId: edu.uci.ics.amber.engine.common.OperatorIdentity, layerName: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity = _root_.edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity( + ): _root_.edu.uci.ics.amber.engine.common.PhysicalOpIdentity = _root_.edu.uci.ics.amber.engine.common.PhysicalOpIdentity( logicalOpId, layerName ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/PortIdentity.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/PortIdentity.scala similarity index 83% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/PortIdentity.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/PortIdentity.scala index e8f98bb680d..a4bb48ce6fa 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/PortIdentity.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/PortIdentity.scala @@ -3,7 +3,7 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflow +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class PortIdentity( @@ -75,13 +75,13 @@ final case class PortIdentity( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflow.PortIdentity.type = edu.uci.ics.amber.engine.common.workflow.PortIdentity + def companion: edu.uci.ics.amber.engine.common.PortIdentity.type = edu.uci.ics.amber.engine.common.PortIdentity // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.PortIdentity]) } -object PortIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflow.PortIdentity] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflow.PortIdentity] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflow.PortIdentity = { +object PortIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.PortIdentity] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.PortIdentity] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.PortIdentity = { var __id: _root_.scala.Int = 0 var __internal: _root_.scala.Boolean = false var _done__ = false @@ -96,15 +96,15 @@ object PortIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber. case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflow.PortIdentity( + edu.uci.ics.amber.engine.common.PortIdentity( id = __id, internal = __internal ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflow.PortIdentity] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.PortIdentity] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflow.PortIdentity( + edu.uci.ics.amber.engine.common.PortIdentity( id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Int]).getOrElse(0), internal = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[_root_.scala.Boolean]).getOrElse(false) ) @@ -115,11 +115,11 @@ object PortIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber. def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflow.PortIdentity( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.PortIdentity( id = 0, internal = false ) - implicit class PortIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PortIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflow.PortIdentity](_l) { + implicit class PortIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.PortIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.PortIdentity](_l) { def id: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Int] = field(_.id)((c_, f_) => c_.copy(id = f_)) def internal: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Boolean] = field(_.internal)((c_, f_) => c_.copy(internal = f_)) } @@ -128,7 +128,7 @@ object PortIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber. def of( id: _root_.scala.Int, internal: _root_.scala.Boolean - ): _root_.edu.uci.ics.amber.engine.common.workflow.PortIdentity = _root_.edu.uci.ics.amber.engine.common.workflow.PortIdentity( + ): _root_.edu.uci.ics.amber.engine.common.PortIdentity = _root_.edu.uci.ics.amber.engine.common.PortIdentity( id, internal ) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/VirtualIdentityProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/VirtualIdentityProto.scala new file mode 100644 index 00000000000..e0f421a47a4 --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/VirtualIdentityProto.scala @@ -0,0 +1,49 @@ +// Generated by the Scala Plugin for the Protocol Buffer Compiler. +// Do not edit! +// +// Protofile syntax: PROTO3 + +package edu.uci.ics.amber.engine.common + +object VirtualIdentityProto extends _root_.scalapb.GeneratedFileObject { + lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( + scalapb.options.ScalapbProto + ) + lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = + Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( + edu.uci.ics.amber.engine.common.WorkflowIdentity, + edu.uci.ics.amber.engine.common.ExecutionIdentity, + edu.uci.ics.amber.engine.common.ActorVirtualIdentity, + edu.uci.ics.amber.engine.common.ChannelIdentity, + edu.uci.ics.amber.engine.common.OperatorIdentity, + edu.uci.ics.amber.engine.common.PhysicalOpIdentity, + edu.uci.ics.amber.engine.common.ChannelMarkerIdentity + ) + private lazy val ProtoBytes: _root_.scala.Array[Byte] = + scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( + """CjZlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3ZpcnR1YWxfaWRlbnRpdHkucHJvdG8SH2VkdS51Y2kuaWNzLmFtY + mVyLmVuZ2luZS5jb21tb24aFXNjYWxhcGIvc2NhbGFwYi5wcm90byIrChBXb3JrZmxvd0lkZW50aXR5EhcKAmlkGAEgASgDQgfiP + wQSAmlkUgJpZCIsChFFeGVjdXRpb25JZGVudGl0eRIXCgJpZBgBIAEoA0IH4j8EEgJpZFICaWQiNQoUQWN0b3JWaXJ0dWFsSWRlb + nRpdHkSHQoEbmFtZRgBIAEoCUIJ4j8GEgRuYW1lUgRuYW1lIqACCg9DaGFubmVsSWRlbnRpdHkScQoOZnJvbV93b3JrZXJfaWQYA + SABKAsyNS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkFjdG9yVmlydHVhbElkZW50aXR5QhTiPxESDGZyb21Xb3JrZ + XJJZPABAVIMZnJvbVdvcmtlcklkEmsKDHRvX3dvcmtlcl9pZBgCIAEoCzI1LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb + 24uQWN0b3JWaXJ0dWFsSWRlbnRpdHlCEuI/DxIKdG9Xb3JrZXJJZPABAVIKdG9Xb3JrZXJJZBItCgppc19jb250cm9sGAMgASgIQ + g7iPwsSCWlzQ29udHJvbFIJaXNDb250cm9sIisKEE9wZXJhdG9ySWRlbnRpdHkSFwoCaWQYASABKAlCB+I/BBICaWRSAmlkIq8BC + hJQaHlzaWNhbE9wSWRlbnRpdHkSagoNbG9naWNhbF9vcF9pZBgBIAEoCzIxLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb + 24uT3BlcmF0b3JJZGVudGl0eUIT4j8QEgtsb2dpY2FsT3BJZPABAVILbG9naWNhbE9wSWQSLQoKbGF5ZXJfbmFtZRgCIAEoCUIO4 + j8LEglsYXllck5hbWVSCWxheWVyTmFtZSIwChVDaGFubmVsTWFya2VySWRlbnRpdHkSFwoCaWQYASABKAlCB+I/BBICaWRSAmlkQ + gviPwgQAUgAWAB4AWIGcHJvdG8z""" + ).mkString) + lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { + val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) + _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) + } + lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { + val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) + com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( + scalapb.options.ScalapbProto.javaDescriptor + )) + } + @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") + def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/WorkflowAggregatedState.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowAggregatedState.scala similarity index 88% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/WorkflowAggregatedState.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowAggregatedState.scala index 0a5da75f488..b1a8cc45828 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/architecture/rpc/controlreturns/WorkflowAggregatedState.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowAggregatedState.scala @@ -3,7 +3,7 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.architecture.rpc.controlreturns +package edu.uci.ics.amber.engine.common sealed abstract class WorkflowAggregatedState(val value: _root_.scala.Int) extends _root_.scalapb.GeneratedEnum { type EnumType = WorkflowAggregatedState @@ -17,8 +17,8 @@ sealed abstract class WorkflowAggregatedState(val value: _root_.scala.Int) exten def isFailed: _root_.scala.Boolean = false def isUnknown: _root_.scala.Boolean = false def isKilled: _root_.scala.Boolean = false - def companion: _root_.scalapb.GeneratedEnumCompanion[WorkflowAggregatedState] = edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState - final def asRecognized: _root_.scala.Option[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.Recognized] = if (isUnrecognized) _root_.scala.None else _root_.scala.Some(this.asInstanceOf[edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.Recognized]) + def companion: _root_.scalapb.GeneratedEnumCompanion[WorkflowAggregatedState] = edu.uci.ics.amber.engine.common.WorkflowAggregatedState + final def asRecognized: _root_.scala.Option[edu.uci.ics.amber.engine.common.WorkflowAggregatedState.Recognized] = if (isUnrecognized) _root_.scala.None else _root_.scala.Some(this.asInstanceOf[edu.uci.ics.amber.engine.common.WorkflowAggregatedState.Recognized]) } object WorkflowAggregatedState extends _root_.scalapb.GeneratedEnumCompanion[WorkflowAggregatedState] { @@ -111,6 +111,6 @@ object WorkflowAggregatedState extends _root_.scalapb.GeneratedEnumCompanion[Wor case 9 => KILLED case __other => Unrecognized(__other) } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.EnumDescriptor = ControlreturnsProto.javaDescriptor.getEnumTypes().get(1) - def scalaDescriptor: _root_.scalapb.descriptors.EnumDescriptor = ControlreturnsProto.scalaDescriptor.enums(1) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.EnumDescriptor = WorkflowMetricsProto.javaDescriptor.getEnumTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.EnumDescriptor = WorkflowMetricsProto.scalaDescriptor.enums(0) } \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/WorkflowFatalError.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowFatalError.scala similarity index 78% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/WorkflowFatalError.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowFatalError.scala index 42b43c4d93b..424411ec4bb 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/WorkflowFatalError.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowFatalError.scala @@ -3,11 +3,11 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.workflowruntimestate +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class WorkflowFatalError( - `type`: edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType = edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.COMPILATION_ERROR, + `type`: edu.uci.ics.amber.engine.common.FatalErrorType = edu.uci.ics.amber.engine.common.FatalErrorType.COMPILATION_ERROR, timestamp: com.google.protobuf.timestamp.Timestamp = com.google.protobuf.timestamp.Timestamp.defaultInstance, message: _root_.scala.Predef.String = "", details: _root_.scala.Predef.String = "", @@ -111,7 +111,7 @@ final case class WorkflowFatalError( } }; } - def withType(__v: edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType): WorkflowFatalError = copy(`type` = __v) + def withType(__v: edu.uci.ics.amber.engine.common.FatalErrorType): WorkflowFatalError = copy(`type` = __v) def withTimestamp(__v: com.google.protobuf.timestamp.Timestamp): WorkflowFatalError = copy(timestamp = __v) def withMessage(__v: _root_.scala.Predef.String): WorkflowFatalError = copy(message = __v) def withDetails(__v: _root_.scala.Predef.String): WorkflowFatalError = copy(details = __v) @@ -157,14 +157,14 @@ final case class WorkflowFatalError( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError.type = edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError + def companion: edu.uci.ics.amber.engine.common.WorkflowFatalError.type = edu.uci.ics.amber.engine.common.WorkflowFatalError // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.WorkflowFatalError]) } -object WorkflowFatalError extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError = { - var __type: edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType = edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.COMPILATION_ERROR +object WorkflowFatalError extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.WorkflowFatalError] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.WorkflowFatalError] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.WorkflowFatalError = { + var __type: edu.uci.ics.amber.engine.common.FatalErrorType = edu.uci.ics.amber.engine.common.FatalErrorType.COMPILATION_ERROR var __timestamp: _root_.scala.Option[com.google.protobuf.timestamp.Timestamp] = _root_.scala.None var __message: _root_.scala.Predef.String = "" var __details: _root_.scala.Predef.String = "" @@ -176,7 +176,7 @@ object WorkflowFatalError extends scalapb.GeneratedMessageCompanion[edu.uci.ics. _tag__ match { case 0 => _done__ = true case 8 => - __type = edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.fromValue(_input__.readEnum()) + __type = edu.uci.ics.amber.engine.common.FatalErrorType.fromValue(_input__.readEnum()) case 18 => __timestamp = _root_.scala.Some(__timestamp.fold(_root_.scalapb.LiteParser.readMessage[com.google.protobuf.timestamp.Timestamp](_input__))(_root_.scalapb.LiteParser.readMessage(_input__, _))) case 26 => @@ -190,7 +190,7 @@ object WorkflowFatalError extends scalapb.GeneratedMessageCompanion[edu.uci.ics. case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError( + edu.uci.ics.amber.engine.common.WorkflowFatalError( `type` = __type, timestamp = __timestamp.getOrElse(com.google.protobuf.timestamp.Timestamp.defaultInstance), message = __message, @@ -199,11 +199,11 @@ object WorkflowFatalError extends scalapb.GeneratedMessageCompanion[edu.uci.ics. workerId = __workerId ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.WorkflowFatalError] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError( - `type` = edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.COMPILATION_ERROR.scalaValueDescriptor).number), + edu.uci.ics.amber.engine.common.WorkflowFatalError( + `type` = edu.uci.ics.amber.engine.common.FatalErrorType.fromValue(__fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scalapb.descriptors.EnumValueDescriptor]).getOrElse(edu.uci.ics.amber.engine.common.FatalErrorType.COMPILATION_ERROR.scalaValueDescriptor).number), timestamp = __fieldsMap.get(scalaDescriptor.findFieldByNumber(2).get).map(_.as[com.google.protobuf.timestamp.Timestamp]).getOrElse(com.google.protobuf.timestamp.Timestamp.defaultInstance), message = __fieldsMap.get(scalaDescriptor.findFieldByNumber(3).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), details = __fieldsMap.get(scalaDescriptor.findFieldByNumber(4).get).map(_.as[_root_.scala.Predef.String]).getOrElse(""), @@ -212,8 +212,8 @@ object WorkflowFatalError extends scalapb.GeneratedMessageCompanion[edu.uci.ics. ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowruntimestateProto.javaDescriptor.getMessageTypes().get(10) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowruntimestateProto.scalaDescriptor.messages(10) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = WorkflowRuntimeStateProto.javaDescriptor.getMessageTypes().get(8) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = WorkflowRuntimeStateProto.scalaDescriptor.messages(8) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = { var __out: _root_.scalapb.GeneratedMessageCompanion[_] = null (__number: @_root_.scala.unchecked) match { @@ -224,19 +224,19 @@ object WorkflowFatalError extends scalapb.GeneratedMessageCompanion[edu.uci.ics. lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = { (__fieldNumber: @_root_.scala.unchecked) match { - case 1 => edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType + case 1 => edu.uci.ics.amber.engine.common.FatalErrorType } } - lazy val defaultInstance = edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError( - `type` = edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType.COMPILATION_ERROR, + lazy val defaultInstance = edu.uci.ics.amber.engine.common.WorkflowFatalError( + `type` = edu.uci.ics.amber.engine.common.FatalErrorType.COMPILATION_ERROR, timestamp = com.google.protobuf.timestamp.Timestamp.defaultInstance, message = "", details = "", operatorId = "", workerId = "" ) - implicit class WorkflowFatalErrorLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError](_l) { - def `type`: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType] = field(_.`type`)((c_, f_) => c_.copy(`type` = f_)) + implicit class WorkflowFatalErrorLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.WorkflowFatalError]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.WorkflowFatalError](_l) { + def `type`: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.FatalErrorType] = field(_.`type`)((c_, f_) => c_.copy(`type` = f_)) def timestamp: _root_.scalapb.lenses.Lens[UpperPB, com.google.protobuf.timestamp.Timestamp] = field(_.timestamp)((c_, f_) => c_.copy(timestamp = f_)) def message: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.message)((c_, f_) => c_.copy(message = f_)) def details: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Predef.String] = field(_.details)((c_, f_) => c_.copy(details = f_)) @@ -247,16 +247,16 @@ object WorkflowFatalError extends scalapb.GeneratedMessageCompanion[edu.uci.ics. final val TIMESTAMP_FIELD_NUMBER = 2 final val MESSAGE_FIELD_NUMBER = 3 final val DETAILS_FIELD_NUMBER = 4 - final val OPERATORID_FIELD_NUMBER = 5 - final val WORKERID_FIELD_NUMBER = 6 + final val OPERATOR_ID_FIELD_NUMBER = 5 + final val WORKER_ID_FIELD_NUMBER = 6 def of( - `type`: edu.uci.ics.amber.engine.common.workflowruntimestate.FatalErrorType, + `type`: edu.uci.ics.amber.engine.common.FatalErrorType, timestamp: com.google.protobuf.timestamp.Timestamp, message: _root_.scala.Predef.String, details: _root_.scala.Predef.String, operatorId: _root_.scala.Predef.String, workerId: _root_.scala.Predef.String - ): _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError = _root_.edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError( + ): _root_.edu.uci.ics.amber.engine.common.WorkflowFatalError = _root_.edu.uci.ics.amber.engine.common.WorkflowFatalError( `type`, timestamp, message, diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/WorkflowIdentity.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowIdentity.scala similarity index 75% rename from core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/WorkflowIdentity.scala rename to core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowIdentity.scala index 5e69915365a..b74f1a4c7b3 100644 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/WorkflowIdentity.scala +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowIdentity.scala @@ -3,7 +3,7 @@ // // Protofile syntax: PROTO3 -package edu.uci.ics.amber.engine.common.virtualidentity +package edu.uci.ics.amber.engine.common @SerialVersionUID(0L) final case class WorkflowIdentity( @@ -55,13 +55,13 @@ final case class WorkflowIdentity( } } def toProtoString: _root_.scala.Predef.String = _root_.scalapb.TextFormat.printToSingleLineUnicodeString(this) - def companion: edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity.type = edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity + def companion: edu.uci.ics.amber.engine.common.WorkflowIdentity.type = edu.uci.ics.amber.engine.common.WorkflowIdentity // @@protoc_insertion_point(GeneratedMessage[edu.uci.ics.amber.engine.common.WorkflowIdentity]) } -object WorkflowIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity] { - implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity] = this - def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity = { +object WorkflowIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.WorkflowIdentity] { + implicit def messageCompanion: scalapb.GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.WorkflowIdentity] = this + def parseFrom(`_input__`: _root_.com.google.protobuf.CodedInputStream): edu.uci.ics.amber.engine.common.WorkflowIdentity = { var __id: _root_.scala.Long = 0L var _done__ = false while (!_done__) { @@ -73,33 +73,33 @@ object WorkflowIdentity extends scalapb.GeneratedMessageCompanion[edu.uci.ics.am case tag => _input__.skipField(tag) } } - edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity( + edu.uci.ics.amber.engine.common.WorkflowIdentity( id = __id ) } - implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity] = _root_.scalapb.descriptors.Reads{ + implicit def messageReads: _root_.scalapb.descriptors.Reads[edu.uci.ics.amber.engine.common.WorkflowIdentity] = _root_.scalapb.descriptors.Reads{ case _root_.scalapb.descriptors.PMessage(__fieldsMap) => _root_.scala.Predef.require(__fieldsMap.keys.forall(_.containingMessage eq scalaDescriptor), "FieldDescriptor does not match message type.") - edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity( + edu.uci.ics.amber.engine.common.WorkflowIdentity( id = __fieldsMap.get(scalaDescriptor.findFieldByNumber(1).get).map(_.as[_root_.scala.Long]).getOrElse(0L) ) case _ => throw new RuntimeException("Expected PMessage") } - def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualidentityProto.javaDescriptor.getMessageTypes().get(0) - def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualidentityProto.scalaDescriptor.messages(0) + def javaDescriptor: _root_.com.google.protobuf.Descriptors.Descriptor = VirtualIdentityProto.javaDescriptor.getMessageTypes().get(0) + def scalaDescriptor: _root_.scalapb.descriptors.Descriptor = VirtualIdentityProto.scalaDescriptor.messages(0) def messageCompanionForFieldNumber(__number: _root_.scala.Int): _root_.scalapb.GeneratedMessageCompanion[_] = throw new MatchError(__number) lazy val nestedMessagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = Seq.empty def enumCompanionForFieldNumber(__fieldNumber: _root_.scala.Int): _root_.scalapb.GeneratedEnumCompanion[_] = throw new MatchError(__fieldNumber) - lazy val defaultInstance = edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity( + lazy val defaultInstance = edu.uci.ics.amber.engine.common.WorkflowIdentity( id = 0L ) - implicit class WorkflowIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity](_l) { + implicit class WorkflowIdentityLens[UpperPB](_l: _root_.scalapb.lenses.Lens[UpperPB, edu.uci.ics.amber.engine.common.WorkflowIdentity]) extends _root_.scalapb.lenses.ObjectLens[UpperPB, edu.uci.ics.amber.engine.common.WorkflowIdentity](_l) { def id: _root_.scalapb.lenses.Lens[UpperPB, _root_.scala.Long] = field(_.id)((c_, f_) => c_.copy(id = f_)) } final val ID_FIELD_NUMBER = 1 def of( id: _root_.scala.Long - ): _root_.edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity = _root_.edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity( + ): _root_.edu.uci.ics.amber.engine.common.WorkflowIdentity = _root_.edu.uci.ics.amber.engine.common.WorkflowIdentity( id ) // @@protoc_insertion_point(GeneratedMessageCompanion[edu.uci.ics.amber.engine.common.WorkflowIdentity]) diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowMetricsProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowMetricsProto.scala new file mode 100644 index 00000000000..7f2071f19fd --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowMetricsProto.scala @@ -0,0 +1,54 @@ +// Generated by the Scala Plugin for the Protocol Buffer Compiler. +// Do not edit! +// +// Protofile syntax: PROTO3 + +package edu.uci.ics.amber.engine.common + +object WorkflowMetricsProto extends _root_.scalapb.GeneratedFileObject { + lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( + edu.uci.ics.amber.engine.architecture.worker.StatisticsProto, + edu.uci.ics.amber.engine.common.VirtualIdentityProto, + scalapb.options.ScalapbProto + ) + lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = + Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( + edu.uci.ics.amber.engine.common.OperatorStatistics, + edu.uci.ics.amber.engine.common.OperatorMetrics + ) + private lazy val ProtoBytes: _root_.scala.Array[Byte] = + scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( + """CjZlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3dvcmtmbG93X21ldHJpY3MucHJvdG8SH2VkdS51Y2kuaWNzLmFtY + mVyLmVuZ2luZS5jb21tb24aPWVkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9hcmNoaXRlY3R1cmUvd29ya2VyL3N0YXRpc3RpY3Muc + HJvdG8aNmVkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9jb21tb24vdmlydHVhbF9pZGVudGl0eS5wcm90bxoVc2NhbGFwYi9zY2FsY + XBiLnByb3RvIoIEChJPcGVyYXRvclN0YXRpc3RpY3MSdQoLaW5wdXRfY291bnQYASADKAsyQy5lZHUudWNpLmljcy5hbWJlci5lb + mdpbmUuYXJjaGl0ZWN0dXJlLndvcmtlci5Qb3J0VHVwbGVDb3VudE1hcHBpbmdCD+I/DBIKaW5wdXRDb3VudFIKaW5wdXRDb3Vud + BJ4CgxvdXRwdXRfY291bnQYAiADKAsyQy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLndvcmtlci5Qb3J0V + HVwbGVDb3VudE1hcHBpbmdCEOI/DRILb3V0cHV0Q291bnRSC291dHB1dENvdW50EjAKC251bV93b3JrZXJzGAMgASgFQg/iPwwSC + m51bVdvcmtlcnNSCm51bVdvcmtlcnMSSQoUZGF0YV9wcm9jZXNzaW5nX3RpbWUYBCABKANCF+I/FBISZGF0YVByb2Nlc3NpbmdUa + W1lUhJkYXRhUHJvY2Vzc2luZ1RpbWUSUgoXY29udHJvbF9wcm9jZXNzaW5nX3RpbWUYBSABKANCGuI/FxIVY29udHJvbFByb2Nlc + 3NpbmdUaW1lUhVjb250cm9sUHJvY2Vzc2luZ1RpbWUSKgoJaWRsZV90aW1lGAYgASgDQg3iPwoSCGlkbGVUaW1lUghpZGxlVGltZ + SKMAgoPT3BlcmF0b3JNZXRyaWNzEnYKDm9wZXJhdG9yX3N0YXRlGAEgASgOMjguZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvb + W1vbi5Xb3JrZmxvd0FnZ3JlZ2F0ZWRTdGF0ZUIV4j8SEg1vcGVyYXRvclN0YXRl8AEBUg1vcGVyYXRvclN0YXRlEoABChNvcGVyY + XRvcl9zdGF0aXN0aWNzGAIgASgLMjMuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5PcGVyYXRvclN0YXRpc3RpY3NCG + uI/FxISb3BlcmF0b3JTdGF0aXN0aWNz8AEBUhJvcGVyYXRvclN0YXRpc3RpY3MqrwIKF1dvcmtmbG93QWdncmVnYXRlZFN0YXRlE + iUKDVVOSU5JVElBTElaRUQQABoS4j8PEg1VTklOSVRJQUxJWkVEEhUKBVJFQURZEAEaCuI/BxIFUkVBRFkSGQoHUlVOTklORxACG + gziPwkSB1JVTk5JTkcSGQoHUEFVU0lORxADGgziPwkSB1BBVVNJTkcSFwoGUEFVU0VEEAQaC+I/CBIGUEFVU0VEEhsKCFJFU1VNS + U5HEAUaDeI/ChIIUkVTVU1JTkcSHQoJQ09NUExFVEVEEAYaDuI/CxIJQ09NUExFVEVEEhcKBkZBSUxFRBAHGgviPwgSBkZBSUxFR + BIZCgdVTktOT1dOEAgaDOI/CRIHVU5LTk9XThIXCgZLSUxMRUQQCRoL4j8IEgZLSUxMRURCC+I/CBABSABYAHgAYgZwcm90bzM=""" + ).mkString) + lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { + val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) + _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) + } + lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { + val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) + com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( + edu.uci.ics.amber.engine.architecture.worker.StatisticsProto.javaDescriptor, + edu.uci.ics.amber.engine.common.VirtualIdentityProto.javaDescriptor, + scalapb.options.ScalapbProto.javaDescriptor + )) + } + @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") + def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowProto.scala new file mode 100644 index 00000000000..8260a011c48 --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowProto.scala @@ -0,0 +1,52 @@ +// Generated by the Scala Plugin for the Protocol Buffer Compiler. +// Do not edit! +// +// Protofile syntax: PROTO3 + +package edu.uci.ics.amber.engine.common + +object WorkflowProto extends _root_.scalapb.GeneratedFileObject { + lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( + edu.uci.ics.amber.engine.common.VirtualIdentityProto, + scalapb.options.ScalapbProto + ) + lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = + Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( + edu.uci.ics.amber.engine.common.PortIdentity, + edu.uci.ics.amber.engine.common.InputPort, + edu.uci.ics.amber.engine.common.OutputPort, + edu.uci.ics.amber.engine.common.PhysicalLink + ) + private lazy val ProtoBytes: _root_.scala.Array[Byte] = + scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( + """Ci5lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3dvcmtmbG93LnByb3RvEh9lZHUudWNpLmljcy5hbWJlci5lbmdpb + mUuY29tbW9uGjZlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3ZpcnR1YWxfaWRlbnRpdHkucHJvdG8aFXNjYWxhcGIvc + 2NhbGFwYi5wcm90byJSCgxQb3J0SWRlbnRpdHkSFwoCaWQYASABKAVCB+I/BBICaWRSAmlkEikKCGludGVybmFsGAIgASgIQg3iP + woSCGludGVybmFsUghpbnRlcm5hbCKzAgoJSW5wdXRQb3J0EkkKAmlkGAEgASgLMi0uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lL + mNvbW1vbi5Qb3J0SWRlbnRpdHlCCuI/BxICaWTwAQFSAmlkEjMKDGRpc3BsYXlfbmFtZRgCIAEoCUIQ4j8NEgtkaXNwbGF5TmFtZ + VILZGlzcGxheU5hbWUSQAoRYWxsb3dfbXVsdGlfbGlua3MYAyABKAhCFOI/ERIPYWxsb3dNdWx0aUxpbmtzUg9hbGxvd011bHRpT + Glua3MSZAoMZGVwZW5kZW5jaWVzGAQgAygLMi0uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5Qb3J0SWRlbnRpdHlCE + eI/DhIMZGVwZW5kZW5jaWVzUgxkZXBlbmRlbmNpZXMitwEKCk91dHB1dFBvcnQSSQoCaWQYASABKAsyLS5lZHUudWNpLmljcy5hb + WJlci5lbmdpbmUuY29tbW9uLlBvcnRJZGVudGl0eUIK4j8HEgJpZPABAVICaWQSMwoMZGlzcGxheV9uYW1lGAIgASgJQhDiPw0SC + 2Rpc3BsYXlOYW1lUgtkaXNwbGF5TmFtZRIpCghibG9ja2luZxgDIAEoCEIN4j8KEghibG9ja2luZ1IIYmxvY2tpbmcilgMKDFBoe + XNpY2FsTGluaxJjCgpmcm9tX29wX2lkGAEgASgLMjMuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5QaHlzaWNhbE9wS + WRlbnRpdHlCEOI/DRIIZnJvbU9wSWTwAQFSCGZyb21PcElkEmMKDGZyb21fcG9ydF9pZBgCIAEoCzItLmVkdS51Y2kuaWNzLmFtY + mVyLmVuZ2luZS5jb21tb24uUG9ydElkZW50aXR5QhLiPw8SCmZyb21Qb3J0SWTwAQFSCmZyb21Qb3J0SWQSXQoIdG9fb3BfaWQYA + yABKAsyMy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLlBoeXNpY2FsT3BJZGVudGl0eUIO4j8LEgZ0b09wSWTwAQFSB + nRvT3BJZBJdCgp0b19wb3J0X2lkGAQgASgLMi0uZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5Qb3J0SWRlbnRpdHlCE + OI/DRIIdG9Qb3J0SWTwAQFSCHRvUG9ydElkQgviPwgQAUgAWAB4AGIGcHJvdG8z""" + ).mkString) + lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { + val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) + _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) + } + lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { + val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) + com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( + edu.uci.ics.amber.engine.common.VirtualIdentityProto.javaDescriptor, + scalapb.options.ScalapbProto.javaDescriptor + )) + } + @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") + def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowRuntimeStateProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowRuntimeStateProto.scala new file mode 100644 index 00000000000..77f992dd4a6 --- /dev/null +++ b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/WorkflowRuntimeStateProto.scala @@ -0,0 +1,98 @@ +// Generated by the Scala Plugin for the Protocol Buffer Compiler. +// Do not edit! +// +// Protofile syntax: PROTO3 + +package edu.uci.ics.amber.engine.common + +object WorkflowRuntimeStateProto extends _root_.scalapb.GeneratedFileObject { + lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( + edu.uci.ics.amber.engine.architecture.rpc.ControlCommandsProto, + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnsProto, + edu.uci.ics.amber.engine.common.WorkflowMetricsProto, + edu.uci.ics.amber.engine.common.VirtualIdentityProto, + com.google.protobuf.timestamp.TimestampProto, + scalapb.options.ScalapbProto + ) + lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = + Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( + edu.uci.ics.amber.engine.common.BreakpointFault, + edu.uci.ics.amber.engine.common.OperatorBreakpoints, + edu.uci.ics.amber.engine.common.ExecutionBreakpointStore, + edu.uci.ics.amber.engine.common.EvaluatedValueList, + edu.uci.ics.amber.engine.common.OperatorConsole, + edu.uci.ics.amber.engine.common.ExecutionConsoleStore, + edu.uci.ics.amber.engine.common.OperatorWorkerMapping, + edu.uci.ics.amber.engine.common.ExecutionStatsStore, + edu.uci.ics.amber.engine.common.WorkflowFatalError, + edu.uci.ics.amber.engine.common.ExecutionMetadataStore + ) + private lazy val ProtoBytes: _root_.scala.Array[Byte] = + scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( + """CjxlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3dvcmtmbG93X3J1bnRpbWVfc3RhdGUucHJvdG8SH2VkdS51Y2kua + WNzLmFtYmVyLmVuZ2luZS5jb21tb24aQGVkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9hcmNoaXRlY3R1cmUvcnBjL2NvbnRyb2xfY + 29tbWFuZHMucHJvdG8aP2VkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9hcmNoaXRlY3R1cmUvcnBjL2NvbnRyb2xfcmV0dXJucy5wc + m90bxo2ZWR1L3VjaS9pY3MvYW1iZXIvZW5naW5lL2NvbW1vbi93b3JrZmxvd19tZXRyaWNzLnByb3RvGjZlZHUvdWNpL2ljcy9hb + WJlci9lbmdpbmUvY29tbW9uL3ZpcnR1YWxfaWRlbnRpdHkucHJvdG8aH2dvb2dsZS9wcm90b2J1Zi90aW1lc3RhbXAucHJvdG8aF + XNjYWxhcGIvc2NhbGFwYi5wcm90byK0AgoPQnJlYWtwb2ludEZhdWx0EjAKC3dvcmtlcl9uYW1lGAEgASgJQg/iPwwSCndvcmtlc + k5hbWVSCndvcmtlck5hbWUSeAoNZmF1bHRlZF90dXBsZRgCIAEoCzJALmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uQ + nJlYWtwb2ludEZhdWx0LkJyZWFrcG9pbnRUdXBsZUIR4j8OEgxmYXVsdGVkVHVwbGVSDGZhdWx0ZWRUdXBsZRp1Cg9CcmVha3Bva + W50VHVwbGUSFwoCaWQYASABKANCB+I/BBICaWRSAmlkEicKCGlzX2lucHV0GAIgASgIQgziPwkSB2lzSW5wdXRSB2lzSW5wdXQSI + AoFdHVwbGUYAyADKAlCCuI/BxIFdHVwbGVSBXR1cGxlIpsBChNPcGVyYXRvckJyZWFrcG9pbnRzEoMBChZ1bnJlc29sdmVkX2JyZ + WFrcG9pbnRzGAEgAygLMjAuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5CcmVha3BvaW50RmF1bHRCGuI/FxIVdW5yZ + XNvbHZlZEJyZWFrcG9pbnRzUhV1bnJlc29sdmVkQnJlYWtwb2ludHMirgIKGEV4ZWN1dGlvbkJyZWFrcG9pbnRTdG9yZRKDAQoNb + 3BlcmF0b3JfaW5mbxgBIAMoCzJLLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uRXhlY3V0aW9uQnJlYWtwb2ludFN0b + 3JlLk9wZXJhdG9ySW5mb0VudHJ5QhHiPw4SDG9wZXJhdG9ySW5mb1IMb3BlcmF0b3JJbmZvGosBChFPcGVyYXRvckluZm9FbnRye + RIaCgNrZXkYASABKAlCCOI/BRIDa2V5UgNrZXkSVgoFdmFsdWUYAiABKAsyNC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tb + W9uLk9wZXJhdG9yQnJlYWtwb2ludHNCCuI/BxIFdmFsdWVSBXZhbHVlOgI4ASJ0ChJFdmFsdWF0ZWRWYWx1ZUxpc3QSXgoGdmFsd + WVzGAEgAygLMjkuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuRXZhbHVhdGVkVmFsdWVCC+I/CBIGd + mFsdWVzUgZ2YWx1ZXMiuwMKD09wZXJhdG9yQ29uc29sZRJ6ChBjb25zb2xlX21lc3NhZ2VzGAEgAygLMjkuZWR1LnVjaS5pY3MuY + W1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuQ29uc29sZU1lc3NhZ2VCFOI/ERIPY29uc29sZU1lc3NhZ2VzUg9jb25zb2xlT + WVzc2FnZXMSlwEKFWV2YWx1YXRlX2V4cHJfcmVzdWx0cxgCIAMoCzJJLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uT + 3BlcmF0b3JDb25zb2xlLkV2YWx1YXRlRXhwclJlc3VsdHNFbnRyeUIY4j8VEhNldmFsdWF0ZUV4cHJSZXN1bHRzUhNldmFsdWF0Z + UV4cHJSZXN1bHRzGpEBChhFdmFsdWF0ZUV4cHJSZXN1bHRzRW50cnkSGgoDa2V5GAEgASgJQgjiPwUSA2tleVIDa2V5ElUKBXZhb + HVlGAIgASgLMjMuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5FdmFsdWF0ZWRWYWx1ZUxpc3RCCuI/BxIFdmFsdWVSB + XZhbHVlOgI4ASKzAgoVRXhlY3V0aW9uQ29uc29sZVN0b3JlEowBChBvcGVyYXRvcl9jb25zb2xlGAEgAygLMksuZWR1LnVjaS5pY + 3MuYW1iZXIuZW5naW5lLmNvbW1vbi5FeGVjdXRpb25Db25zb2xlU3RvcmUuT3BlcmF0b3JDb25zb2xlRW50cnlCFOI/ERIPb3Blc + mF0b3JDb25zb2xlUg9vcGVyYXRvckNvbnNvbGUaigEKFE9wZXJhdG9yQ29uc29sZUVudHJ5EhoKA2tleRgBIAEoCUII4j8FEgNrZ + XlSA2tleRJSCgV2YWx1ZRgCIAEoCzIwLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uT3BlcmF0b3JDb25zb2xlQgriP + wcSBXZhbHVlUgV2YWx1ZToCOAEieAoVT3BlcmF0b3JXb3JrZXJNYXBwaW5nEjAKC29wZXJhdG9yX2lkGAEgASgJQg/iPwwSCm9wZ + XJhdG9ySWRSCm9wZXJhdG9ySWQSLQoKd29ya2VyX2lkcxgCIAMoCUIO4j8LEgl3b3JrZXJJZHNSCXdvcmtlcklkcyKiBAoTRXhlY + 3V0aW9uU3RhdHNTdG9yZRI8Cg9zdGFydF90aW1lc3RhbXAYASABKANCE+I/EBIOc3RhcnRUaW1lc3RhbXBSDnN0YXJ0VGltZXN0Y + W1wEjYKDWVuZF90aW1lc3RhbXAYAiABKANCEeI/DhIMZW5kVGltZXN0YW1wUgxlbmRUaW1lc3RhbXASfgoNb3BlcmF0b3JfaW5mb + xgDIAMoCzJGLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uRXhlY3V0aW9uU3RhdHNTdG9yZS5PcGVyYXRvckluZm9Fb + nRyeUIR4j8OEgxvcGVyYXRvckluZm9SDG9wZXJhdG9ySW5mbxKKAQoXb3BlcmF0b3Jfd29ya2VyX21hcHBpbmcYBCADKAsyNi5lZ + HUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLk9wZXJhdG9yV29ya2VyTWFwcGluZ0Ia4j8XEhVvcGVyYXRvcldvcmtlck1hc + HBpbmdSFW9wZXJhdG9yV29ya2VyTWFwcGluZxqHAQoRT3BlcmF0b3JJbmZvRW50cnkSGgoDa2V5GAEgASgJQgjiPwUSA2tleVIDa + 2V5ElIKBXZhbHVlGAIgASgLMjAuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5PcGVyYXRvck1ldHJpY3NCCuI/BxIFd + mFsdWVSBXZhbHVlOgI4ASLfAgoSV29ya2Zsb3dGYXRhbEVycm9yEk4KBHR5cGUYASABKA4yLy5lZHUudWNpLmljcy5hbWJlci5lb + mdpbmUuY29tbW9uLkZhdGFsRXJyb3JUeXBlQgniPwYSBHR5cGVSBHR5cGUSSwoJdGltZXN0YW1wGAIgASgLMhouZ29vZ2xlLnByb + 3RvYnVmLlRpbWVzdGFtcEIR4j8OEgl0aW1lc3RhbXDwAQFSCXRpbWVzdGFtcBImCgdtZXNzYWdlGAMgASgJQgziPwkSB21lc3NhZ + 2VSB21lc3NhZ2USJgoHZGV0YWlscxgEIAEoCUIM4j8JEgdkZXRhaWxzUgdkZXRhaWxzEjAKC29wZXJhdG9yX2lkGAUgASgJQg/iP + wwSCm9wZXJhdG9ySWRSCm9wZXJhdG9ySWQSKgoJd29ya2VyX2lkGAYgASgJQg3iPwoSCHdvcmtlcklkUgh3b3JrZXJJZCKCAwoWR + XhlY3V0aW9uTWV0YWRhdGFTdG9yZRJaCgVzdGF0ZRgBIAEoDjI4LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uV29ya + 2Zsb3dBZ2dyZWdhdGVkU3RhdGVCCuI/BxIFc3RhdGVSBXN0YXRlEmgKDGZhdGFsX2Vycm9ycxgCIAMoCzIzLmVkdS51Y2kuaWNzL + mFtYmVyLmVuZ2luZS5jb21tb24uV29ya2Zsb3dGYXRhbEVycm9yQhDiPw0SC2ZhdGFsRXJyb3JzUgtmYXRhbEVycm9ycxJqCgxle + GVjdXRpb25faWQYAyABKAsyMi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkV4ZWN1dGlvbklkZW50aXR5QhPiPxASC + 2V4ZWN1dGlvbklk8AEBUgtleGVjdXRpb25JZBI2Cg1pc19yZWNvdmVyaW5nGAQgASgIQhHiPw4SDGlzUmVjb3ZlcmluZ1IMaXNSZ + WNvdmVyaW5nKm4KDkZhdGFsRXJyb3JUeXBlEi0KEUNPTVBJTEFUSU9OX0VSUk9SEAAaFuI/ExIRQ09NUElMQVRJT05fRVJST1ISL + QoRRVhFQ1VUSU9OX0ZBSUxVUkUQARoW4j8TEhFFWEVDVVRJT05fRkFJTFVSRUIL4j8IEAFIAFgAeABiBnByb3RvMw==""" + ).mkString) + lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { + val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) + _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) + } + lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { + val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) + com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( + edu.uci.ics.amber.engine.architecture.rpc.ControlCommandsProto.javaDescriptor, + edu.uci.ics.amber.engine.architecture.rpc.ControlReturnsProto.javaDescriptor, + edu.uci.ics.amber.engine.common.WorkflowMetricsProto.javaDescriptor, + edu.uci.ics.amber.engine.common.VirtualIdentityProto.javaDescriptor, + com.google.protobuf.timestamp.TimestampProto.javaDescriptor, + scalapb.options.ScalapbProto.javaDescriptor + )) + } + @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") + def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor +} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/AmbermessageProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/AmbermessageProto.scala deleted file mode 100644 index d0fda2f5631..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/ambermessage/AmbermessageProto.scala +++ /dev/null @@ -1,53 +0,0 @@ -// Generated by the Scala Plugin for the Protocol Buffer Compiler. -// Do not edit! -// -// Protofile syntax: PROTO3 - -package edu.uci.ics.amber.engine.common.ambermessage - -object AmbermessageProto extends _root_.scalapb.GeneratedFileObject { - lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlcommandsProto, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlreturnsProto, - edu.uci.ics.amber.engine.common.virtualidentity.VirtualidentityProto, - scalapb.options.ScalapbProto - ) - lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = - Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - edu.uci.ics.amber.engine.common.ambermessage.ControlPayloadV2, - edu.uci.ics.amber.engine.common.ambermessage.PythonDataHeader, - edu.uci.ics.amber.engine.common.ambermessage.PythonControlMessage - ) - private lazy val ProtoBytes: _root_.scala.Array[Byte] = - scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( - """CjJlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL2FtYmVybWVzc2FnZS5wcm90bxIfZWR1LnVjaS5pY3MuYW1iZXIuZ - W5naW5lLmNvbW1vbho/ZWR1L3VjaS9pY3MvYW1iZXIvZW5naW5lL2FyY2hpdGVjdHVyZS9ycGMvY29udHJvbGNvbW1hbmRzLnByb - 3RvGj5lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy9jb250cm9scmV0dXJucy5wcm90bxo1ZWR1L3Vja - S9pY3MvYW1iZXIvZW5naW5lL2NvbW1vbi92aXJ0dWFsaWRlbnRpdHkucHJvdG8aFXNjYWxhcGIvc2NhbGFwYi5wcm90byKnAgoQQ - 29udHJvbFBheWxvYWRWMhKFAQoSY29udHJvbF9pbnZvY2F0aW9uGAEgASgLMjwuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY - 2hpdGVjdHVyZS5ycGMuQ29udHJvbEludm9jYXRpb25CFuI/ExIRY29udHJvbEludm9jYXRpb25IAFIRY29udHJvbEludm9jYXRpb - 24SgQEKEXJldHVybl9pbnZvY2F0aW9uGAIgASgLMjsuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS5ycGMuU - mV0dXJuSW52b2NhdGlvbkIV4j8SEhByZXR1cm5JbnZvY2F0aW9uSABSEHJldHVybkludm9jYXRpb25CBwoFdmFsdWUinQEKEFB5d - GhvbkRhdGFIZWFkZXISVAoDdGFnGAEgASgLMjUuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5BY3RvclZpcnR1YWxJZ - GVudGl0eUIL4j8IEgN0YWfwAQFSA3RhZxIzCgxwYXlsb2FkX3R5cGUYAiABKAlCEOI/DRILcGF5bG9hZFR5cGVSC3BheWxvYWRUe - XBlIsoBChRQeXRob25Db250cm9sTWVzc2FnZRJUCgN0YWcYASABKAsyNS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uL - kFjdG9yVmlydHVhbElkZW50aXR5QgviPwgSA3RhZ/ABAVIDdGFnElwKB3BheWxvYWQYAiABKAsyMS5lZHUudWNpLmljcy5hbWJlc - i5lbmdpbmUuY29tbW9uLkNvbnRyb2xQYXlsb2FkVjJCD+I/DBIHcGF5bG9hZPABAVIHcGF5bG9hZEIJ4j8GSABYAHgBYgZwcm90b - zM=""" - ).mkString) - lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { - val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) - _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) - } - lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { - val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) - com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlcommandsProto.javaDescriptor, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlreturnsProto.javaDescriptor, - edu.uci.ics.amber.engine.common.virtualidentity.VirtualidentityProto.javaDescriptor, - scalapb.options.ScalapbProto.javaDescriptor - )) - } - @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") - def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor -} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/VirtualidentityProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/VirtualidentityProto.scala deleted file mode 100644 index 7f75701f4d7..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/virtualidentity/VirtualidentityProto.scala +++ /dev/null @@ -1,49 +0,0 @@ -// Generated by the Scala Plugin for the Protocol Buffer Compiler. -// Do not edit! -// -// Protofile syntax: PROTO3 - -package edu.uci.ics.amber.engine.common.virtualidentity - -object VirtualidentityProto extends _root_.scalapb.GeneratedFileObject { - lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( - scalapb.options.ScalapbProto - ) - lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = - Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - edu.uci.ics.amber.engine.common.virtualidentity.WorkflowIdentity, - edu.uci.ics.amber.engine.common.virtualidentity.ExecutionIdentity, - edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity, - edu.uci.ics.amber.engine.common.virtualidentity.ChannelIdentity, - edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity, - edu.uci.ics.amber.engine.common.virtualidentity.PhysicalOpIdentity, - edu.uci.ics.amber.engine.common.virtualidentity.ChannelMarkerIdentity - ) - private lazy val ProtoBytes: _root_.scala.Array[Byte] = - scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( - """CjVlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3ZpcnR1YWxpZGVudGl0eS5wcm90bxIfZWR1LnVjaS5pY3MuYW1iZ - XIuZW5naW5lLmNvbW1vbhoVc2NhbGFwYi9zY2FsYXBiLnByb3RvIisKEFdvcmtmbG93SWRlbnRpdHkSFwoCaWQYASABKANCB+I/B - BICaWRSAmlkIiwKEUV4ZWN1dGlvbklkZW50aXR5EhcKAmlkGAEgASgDQgfiPwQSAmlkUgJpZCI1ChRBY3RvclZpcnR1YWxJZGVud - Gl0eRIdCgRuYW1lGAEgASgJQgniPwYSBG5hbWVSBG5hbWUimwIKD0NoYW5uZWxJZGVudGl0eRJvCgxmcm9tV29ya2VySWQYASABK - AsyNS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkFjdG9yVmlydHVhbElkZW50aXR5QhTiPxESDGZyb21Xb3JrZXJJZ - PABAVIMZnJvbVdvcmtlcklkEmkKCnRvV29ya2VySWQYAiABKAsyNS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkFjd - G9yVmlydHVhbElkZW50aXR5QhLiPw8SCnRvV29ya2VySWTwAQFSCnRvV29ya2VySWQSLAoJaXNDb250cm9sGAMgASgIQg7iPwsSC - WlzQ29udHJvbFIJaXNDb250cm9sIisKEE9wZXJhdG9ySWRlbnRpdHkSFwoCaWQYASABKAlCB+I/BBICaWRSAmlkIqwBChJQaHlza - WNhbE9wSWRlbnRpdHkSaAoLbG9naWNhbE9wSWQYASABKAsyMS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLk9wZXJhd - G9ySWRlbnRpdHlCE+I/EBILbG9naWNhbE9wSWTwAQFSC2xvZ2ljYWxPcElkEiwKCWxheWVyTmFtZRgCIAEoCUIO4j8LEglsYXllc - k5hbWVSCWxheWVyTmFtZSIwChVDaGFubmVsTWFya2VySWRlbnRpdHkSFwoCaWQYASABKAlCB+I/BBICaWRSAmlkQgniPwZIAFgAe - AFiBnByb3RvMw==""" - ).mkString) - lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { - val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) - _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) - } - lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { - val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) - com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( - scalapb.options.ScalapbProto.javaDescriptor - )) - } - @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") - def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor -} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/WorkflowProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/WorkflowProto.scala deleted file mode 100644 index 2d092df178b..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflow/WorkflowProto.scala +++ /dev/null @@ -1,52 +0,0 @@ -// Generated by the Scala Plugin for the Protocol Buffer Compiler. -// Do not edit! -// -// Protofile syntax: PROTO3 - -package edu.uci.ics.amber.engine.common.workflow - -object WorkflowProto extends _root_.scalapb.GeneratedFileObject { - lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( - edu.uci.ics.amber.engine.common.virtualidentity.VirtualidentityProto, - scalapb.options.ScalapbProto - ) - lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = - Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - edu.uci.ics.amber.engine.common.workflow.PortIdentity, - edu.uci.ics.amber.engine.common.workflow.InputPort, - edu.uci.ics.amber.engine.common.workflow.OutputPort, - edu.uci.ics.amber.engine.common.workflow.PhysicalLink - ) - private lazy val ProtoBytes: _root_.scala.Array[Byte] = - scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( - """Ci5lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3dvcmtmbG93LnByb3RvEh9lZHUudWNpLmljcy5hbWJlci5lbmdpb - mUuY29tbW9uGjVlZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3ZpcnR1YWxpZGVudGl0eS5wcm90bxoVc2NhbGFwYi9zY - 2FsYXBiLnByb3RvIlIKDFBvcnRJZGVudGl0eRIXCgJpZBgBIAEoBUIH4j8EEgJpZFICaWQSKQoIaW50ZXJuYWwYAiABKAhCDeI/C - hIIaW50ZXJuYWxSCGludGVybmFsIrACCglJbnB1dFBvcnQSSQoCaWQYASABKAsyLS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY - 29tbW9uLlBvcnRJZGVudGl0eUIK4j8HEgJpZPABAVICaWQSMgoLZGlzcGxheU5hbWUYAiABKAlCEOI/DRILZGlzcGxheU5hbWVSC - 2Rpc3BsYXlOYW1lEj4KD2FsbG93TXVsdGlMaW5rcxgDIAEoCEIU4j8REg9hbGxvd011bHRpTGlua3NSD2FsbG93TXVsdGlMaW5rc - xJkCgxkZXBlbmRlbmNpZXMYBCADKAsyLS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLlBvcnRJZGVudGl0eUIR4j8OE - gxkZXBlbmRlbmNpZXNSDGRlcGVuZGVuY2llcyK2AQoKT3V0cHV0UG9ydBJJCgJpZBgBIAEoCzItLmVkdS51Y2kuaWNzLmFtYmVyL - mVuZ2luZS5jb21tb24uUG9ydElkZW50aXR5QgriPwcSAmlk8AEBUgJpZBIyCgtkaXNwbGF5TmFtZRgCIAEoCUIQ4j8NEgtkaXNwb - GF5TmFtZVILZGlzcGxheU5hbWUSKQoIYmxvY2tpbmcYAyABKAhCDeI/ChIIYmxvY2tpbmdSCGJsb2NraW5nIo4DCgxQaHlzaWNhb - ExpbmsSYQoIZnJvbU9wSWQYASABKAsyMy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLlBoeXNpY2FsT3BJZGVudGl0e - UIQ4j8NEghmcm9tT3BJZPABAVIIZnJvbU9wSWQSYQoKZnJvbVBvcnRJZBgCIAEoCzItLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZ - S5jb21tb24uUG9ydElkZW50aXR5QhLiPw8SCmZyb21Qb3J0SWTwAQFSCmZyb21Qb3J0SWQSWwoGdG9PcElkGAMgASgLMjMuZWR1L - nVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5QaHlzaWNhbE9wSWRlbnRpdHlCDuI/CxIGdG9PcElk8AEBUgZ0b09wSWQSWwoId - G9Qb3J0SWQYBCABKAsyLS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLlBvcnRJZGVudGl0eUIQ4j8NEgh0b1BvcnRJZ - PABAVIIdG9Qb3J0SWRCCeI/BkgAWAB4AGIGcHJvdG8z""" - ).mkString) - lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { - val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) - _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) - } - lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { - val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) - com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( - edu.uci.ics.amber.engine.common.virtualidentity.VirtualidentityProto.javaDescriptor, - scalapb.options.ScalapbProto.javaDescriptor - )) - } - @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") - def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor -} \ No newline at end of file diff --git a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/WorkflowruntimestateProto.scala b/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/WorkflowruntimestateProto.scala deleted file mode 100644 index a0774a609a2..00000000000 --- a/core/amber/src/main/scalapb/edu/uci/ics/amber/engine/common/workflowruntimestate/WorkflowruntimestateProto.scala +++ /dev/null @@ -1,111 +0,0 @@ -// Generated by the Scala Plugin for the Protocol Buffer Compiler. -// Do not edit! -// -// Protofile syntax: PROTO3 - -package edu.uci.ics.amber.engine.common.workflowruntimestate - -object WorkflowruntimestateProto extends _root_.scalapb.GeneratedFileObject { - lazy val dependencies: Seq[_root_.scalapb.GeneratedFileObject] = Seq( - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlcommandsProto, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlreturnsProto, - edu.uci.ics.amber.engine.architecture.worker.statistics.StatisticsProto, - edu.uci.ics.amber.engine.common.virtualidentity.VirtualidentityProto, - com.google.protobuf.timestamp.TimestampProto, - scalapb.options.ScalapbProto - ) - lazy val messagesCompanions: Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]] = - Seq[_root_.scalapb.GeneratedMessageCompanion[_ <: _root_.scalapb.GeneratedMessage]]( - edu.uci.ics.amber.engine.common.workflowruntimestate.BreakpointFault, - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorBreakpoints, - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionBreakpointStore, - edu.uci.ics.amber.engine.common.workflowruntimestate.EvaluatedValueList, - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorConsole, - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionConsoleStore, - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorWorkerMapping, - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorStatistics, - edu.uci.ics.amber.engine.common.workflowruntimestate.OperatorMetrics, - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionStatsStore, - edu.uci.ics.amber.engine.common.workflowruntimestate.WorkflowFatalError, - edu.uci.ics.amber.engine.common.workflowruntimestate.ExecutionMetadataStore - ) - private lazy val ProtoBytes: _root_.scala.Array[Byte] = - scalapb.Encoding.fromBase64(scala.collection.immutable.Seq( - """CjplZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvY29tbW9uL3dvcmtmbG93cnVudGltZXN0YXRlLnByb3RvEh9lZHUudWNpLmljc - y5hbWJlci5lbmdpbmUuY29tbW9uGj9lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3JwYy9jb250cm9sY29tb - WFuZHMucHJvdG8aPmVkdS91Y2kvaWNzL2FtYmVyL2VuZ2luZS9hcmNoaXRlY3R1cmUvcnBjL2NvbnRyb2xyZXR1cm5zLnByb3RvG - j1lZHUvdWNpL2ljcy9hbWJlci9lbmdpbmUvYXJjaGl0ZWN0dXJlL3dvcmtlci9zdGF0aXN0aWNzLnByb3RvGjVlZHUvdWNpL2ljc - y9hbWJlci9lbmdpbmUvY29tbW9uL3ZpcnR1YWxpZGVudGl0eS5wcm90bxofZ29vZ2xlL3Byb3RvYnVmL3RpbWVzdGFtcC5wcm90b - xoVc2NhbGFwYi9zY2FsYXBiLnByb3RvIrQCCg9CcmVha3BvaW50RmF1bHQSMAoLd29ya2VyX25hbWUYASABKAlCD+I/DBIKd29ya - 2VyTmFtZVIKd29ya2VyTmFtZRJ4Cg1mYXVsdGVkX3R1cGxlGAIgASgLMkAuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vb - i5CcmVha3BvaW50RmF1bHQuQnJlYWtwb2ludFR1cGxlQhHiPw4SDGZhdWx0ZWRUdXBsZVIMZmF1bHRlZFR1cGxlGnUKD0JyZWFrc - G9pbnRUdXBsZRIXCgJpZBgBIAEoA0IH4j8EEgJpZFICaWQSJwoIaXNfaW5wdXQYAiABKAhCDOI/CRIHaXNJbnB1dFIHaXNJbnB1d - BIgCgV0dXBsZRgDIAMoCUIK4j8HEgV0dXBsZVIFdHVwbGUimwEKE09wZXJhdG9yQnJlYWtwb2ludHMSgwEKFnVucmVzb2x2ZWRfY - nJlYWtwb2ludHMYASADKAsyMC5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkJyZWFrcG9pbnRGYXVsdEIa4j8XEhV1b - nJlc29sdmVkQnJlYWtwb2ludHNSFXVucmVzb2x2ZWRCcmVha3BvaW50cyKuAgoYRXhlY3V0aW9uQnJlYWtwb2ludFN0b3JlEoMBC - g1vcGVyYXRvcl9pbmZvGAEgAygLMksuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5FeGVjdXRpb25CcmVha3BvaW50U - 3RvcmUuT3BlcmF0b3JJbmZvRW50cnlCEeI/DhIMb3BlcmF0b3JJbmZvUgxvcGVyYXRvckluZm8aiwEKEU9wZXJhdG9ySW5mb0Vud - HJ5EhoKA2tleRgBIAEoCUII4j8FEgNrZXlSA2tleRJWCgV2YWx1ZRgCIAEoCzI0LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb - 21tb24uT3BlcmF0b3JCcmVha3BvaW50c0IK4j8HEgV2YWx1ZVIFdmFsdWU6AjgBInQKEkV2YWx1YXRlZFZhbHVlTGlzdBJeCgZ2Y - Wx1ZXMYASADKAsyOS5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5FdmFsdWF0ZWRWYWx1ZUIL4j8IE - gZ2YWx1ZXNSBnZhbHVlcyK7AwoPT3BlcmF0b3JDb25zb2xlEnoKEGNvbnNvbGVfbWVzc2FnZXMYASADKAsyOS5lZHUudWNpLmljc - y5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Db25zb2xlTWVzc2FnZUIU4j8REg9jb25zb2xlTWVzc2FnZXNSD2NvbnNvb - GVNZXNzYWdlcxKXAQoVZXZhbHVhdGVfZXhwcl9yZXN1bHRzGAIgAygLMkkuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vb - i5PcGVyYXRvckNvbnNvbGUuRXZhbHVhdGVFeHByUmVzdWx0c0VudHJ5QhjiPxUSE2V2YWx1YXRlRXhwclJlc3VsdHNSE2V2YWx1Y - XRlRXhwclJlc3VsdHMakQEKGEV2YWx1YXRlRXhwclJlc3VsdHNFbnRyeRIaCgNrZXkYASABKAlCCOI/BRIDa2V5UgNrZXkSVQoFd - mFsdWUYAiABKAsyMy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkV2YWx1YXRlZFZhbHVlTGlzdEIK4j8HEgV2YWx1Z - VIFdmFsdWU6AjgBIrMCChVFeGVjdXRpb25Db25zb2xlU3RvcmUSjAEKEG9wZXJhdG9yX2NvbnNvbGUYASADKAsySy5lZHUudWNpL - mljcy5hbWJlci5lbmdpbmUuY29tbW9uLkV4ZWN1dGlvbkNvbnNvbGVTdG9yZS5PcGVyYXRvckNvbnNvbGVFbnRyeUIU4j8REg9vc - GVyYXRvckNvbnNvbGVSD29wZXJhdG9yQ29uc29sZRqKAQoUT3BlcmF0b3JDb25zb2xlRW50cnkSGgoDa2V5GAEgASgJQgjiPwUSA - 2tleVIDa2V5ElIKBXZhbHVlGAIgASgLMjAuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5PcGVyYXRvckNvbnNvbGVCC - uI/BxIFdmFsdWVSBXZhbHVlOgI4ASJ2ChVPcGVyYXRvcldvcmtlck1hcHBpbmcSLwoKb3BlcmF0b3JJZBgBIAEoCUIP4j8MEgpvc - GVyYXRvcklkUgpvcGVyYXRvcklkEiwKCXdvcmtlcklkcxgCIAMoCUIO4j8LEgl3b3JrZXJJZHNSCXdvcmtlcklkcyKCBAoST3Blc - mF0b3JTdGF0aXN0aWNzEnUKC2lucHV0X2NvdW50GAEgAygLMkMuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZ - S53b3JrZXIuUG9ydFR1cGxlQ291bnRNYXBwaW5nQg/iPwwSCmlucHV0Q291bnRSCmlucHV0Q291bnQSeAoMb3V0cHV0X2NvdW50G - AIgAygLMkMuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmFyY2hpdGVjdHVyZS53b3JrZXIuUG9ydFR1cGxlQ291bnRNYXBwaW5nQ - hDiPw0SC291dHB1dENvdW50UgtvdXRwdXRDb3VudBIwCgtudW1fd29ya2VycxgDIAEoBUIP4j8MEgpudW1Xb3JrZXJzUgpudW1Xb - 3JrZXJzEkkKFGRhdGFfcHJvY2Vzc2luZ190aW1lGAQgASgDQhfiPxQSEmRhdGFQcm9jZXNzaW5nVGltZVISZGF0YVByb2Nlc3Npb - mdUaW1lElIKF2NvbnRyb2xfcHJvY2Vzc2luZ190aW1lGAUgASgDQhriPxcSFWNvbnRyb2xQcm9jZXNzaW5nVGltZVIVY29udHJvb - FByb2Nlc3NpbmdUaW1lEioKCWlkbGVfdGltZRgGIAEoA0IN4j8KEghpZGxlVGltZVIIaWRsZVRpbWUilwIKD09wZXJhdG9yTWV0c - mljcxKAAQoOb3BlcmF0b3Jfc3RhdGUYASABKA4yQi5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuYXJjaGl0ZWN0dXJlLnJwYy5Xb - 3JrZmxvd0FnZ3JlZ2F0ZWRTdGF0ZUIV4j8SEg1vcGVyYXRvclN0YXRl8AEBUg1vcGVyYXRvclN0YXRlEoABChNvcGVyYXRvcl9zd - GF0aXN0aWNzGAIgASgLMjMuZWR1LnVjaS5pY3MuYW1iZXIuZW5naW5lLmNvbW1vbi5PcGVyYXRvclN0YXRpc3RpY3NCGuI/FxISb - 3BlcmF0b3JTdGF0aXN0aWNz8AEBUhJvcGVyYXRvclN0YXRpc3RpY3MioAQKE0V4ZWN1dGlvblN0YXRzU3RvcmUSOwoOc3RhcnRUa - W1lU3RhbXAYASABKANCE+I/EBIOc3RhcnRUaW1lU3RhbXBSDnN0YXJ0VGltZVN0YW1wEjUKDGVuZFRpbWVTdGFtcBgCIAEoA0IR4 - j8OEgxlbmRUaW1lU3RhbXBSDGVuZFRpbWVTdGFtcBJ+Cg1vcGVyYXRvcl9pbmZvGAMgAygLMkYuZWR1LnVjaS5pY3MuYW1iZXIuZ - W5naW5lLmNvbW1vbi5FeGVjdXRpb25TdGF0c1N0b3JlLk9wZXJhdG9ySW5mb0VudHJ5QhHiPw4SDG9wZXJhdG9ySW5mb1IMb3Blc - mF0b3JJbmZvEooBChdvcGVyYXRvcl93b3JrZXJfbWFwcGluZxgEIAMoCzI2LmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb - 24uT3BlcmF0b3JXb3JrZXJNYXBwaW5nQhriPxcSFW9wZXJhdG9yV29ya2VyTWFwcGluZ1IVb3BlcmF0b3JXb3JrZXJNYXBwaW5nG - ocBChFPcGVyYXRvckluZm9FbnRyeRIaCgNrZXkYASABKAlCCOI/BRIDa2V5UgNrZXkSUgoFdmFsdWUYAiABKAsyMC5lZHUudWNpL - mljcy5hbWJlci5lbmdpbmUuY29tbW9uLk9wZXJhdG9yTWV0cmljc0IK4j8HEgV2YWx1ZVIFdmFsdWU6AjgBIt0CChJXb3JrZmxvd - 0ZhdGFsRXJyb3ISTgoEdHlwZRgBIAEoDjIvLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5jb21tb24uRmF0YWxFcnJvclR5cGVCC - eI/BhIEdHlwZVIEdHlwZRJLCgl0aW1lc3RhbXAYAiABKAsyGi5nb29nbGUucHJvdG9idWYuVGltZXN0YW1wQhHiPw4SCXRpbWVzd - GFtcPABAVIJdGltZXN0YW1wEiYKB21lc3NhZ2UYAyABKAlCDOI/CRIHbWVzc2FnZVIHbWVzc2FnZRImCgdkZXRhaWxzGAQgASgJQ - gziPwkSB2RldGFpbHNSB2RldGFpbHMSLwoKb3BlcmF0b3JJZBgFIAEoCUIP4j8MEgpvcGVyYXRvcklkUgpvcGVyYXRvcklkEikKC - HdvcmtlcklkGAYgASgJQg3iPwoSCHdvcmtlcklkUgh3b3JrZXJJZCKLAwoWRXhlY3V0aW9uTWV0YWRhdGFTdG9yZRJkCgVzdGF0Z - RgBIAEoDjJCLmVkdS51Y2kuaWNzLmFtYmVyLmVuZ2luZS5hcmNoaXRlY3R1cmUucnBjLldvcmtmbG93QWdncmVnYXRlZFN0YXRlQ - griPwcSBXN0YXRlUgVzdGF0ZRJoCgxmYXRhbF9lcnJvcnMYAiADKAsyMy5lZHUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uL - ldvcmtmbG93RmF0YWxFcnJvckIQ4j8NEgtmYXRhbEVycm9yc1ILZmF0YWxFcnJvcnMSaQoLZXhlY3V0aW9uSWQYAyABKAsyMi5lZ - HUudWNpLmljcy5hbWJlci5lbmdpbmUuY29tbW9uLkV4ZWN1dGlvbklkZW50aXR5QhPiPxASC2V4ZWN1dGlvbklk8AEBUgtleGVjd - XRpb25JZBI2Cg1pc19yZWNvdmVyaW5nGAQgASgIQhHiPw4SDGlzUmVjb3ZlcmluZ1IMaXNSZWNvdmVyaW5nKm4KDkZhdGFsRXJyb - 3JUeXBlEi0KEUNPTVBJTEFUSU9OX0VSUk9SEAAaFuI/ExIRQ09NUElMQVRJT05fRVJST1ISLQoRRVhFQ1VUSU9OX0ZBSUxVUkUQA - RoW4j8TEhFFWEVDVVRJT05fRkFJTFVSRUIJ4j8GSABYAHgAYgZwcm90bzM=""" - ).mkString) - lazy val scalaDescriptor: _root_.scalapb.descriptors.FileDescriptor = { - val scalaProto = com.google.protobuf.descriptor.FileDescriptorProto.parseFrom(ProtoBytes) - _root_.scalapb.descriptors.FileDescriptor.buildFrom(scalaProto, dependencies.map(_.scalaDescriptor)) - } - lazy val javaDescriptor: com.google.protobuf.Descriptors.FileDescriptor = { - val javaProto = com.google.protobuf.DescriptorProtos.FileDescriptorProto.parseFrom(ProtoBytes) - com.google.protobuf.Descriptors.FileDescriptor.buildFrom(javaProto, _root_.scala.Array( - edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlcommandsProto.javaDescriptor, - edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ControlreturnsProto.javaDescriptor, - edu.uci.ics.amber.engine.architecture.worker.statistics.StatisticsProto.javaDescriptor, - edu.uci.ics.amber.engine.common.virtualidentity.VirtualidentityProto.javaDescriptor, - com.google.protobuf.timestamp.TimestampProto.javaDescriptor, - scalapb.options.ScalapbProto.javaDescriptor - )) - } - @deprecated("Use javaDescriptor instead. In a future version this will refer to scalaDescriptor.", "ScalaPB 0.5.47") - def descriptor: com.google.protobuf.Descriptors.FileDescriptor = javaDescriptor -} \ No newline at end of file diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/TrivialControlSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/TrivialControlSpec.scala index 3a2f09ce01c..64759fda08f 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/TrivialControlSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/TrivialControlSpec.scala @@ -9,13 +9,13 @@ import edu.uci.ics.amber.engine.architecture.common.WorkflowActor.{ RegisterActorRef } import edu.uci.ics.amber.engine.architecture.control.utils.TrivialControlTester -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands._ -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.{ +import edu.uci.ics.amber.engine.architecture.rpc._ +import edu.uci.ics.amber.engine.architecture.rpc.{ IntResponse, ReturnInvocation, StringResponse } -import edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.{ +import edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.{ METHOD_SEND_CHAIN, METHOD_SEND_COLLECT, METHOD_SEND_ERROR_COMMAND, @@ -27,8 +27,8 @@ import edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterGrpc.{ import edu.uci.ics.amber.engine.common.ambermessage.WorkflowMessage.getInMemSize import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient.ControlInvocation -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} -import edu.uci.ics.amber.engine.common.virtualidentity.util.CONTROLLER +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.util.CONTROLLER import io.grpc.MethodDescriptor import org.scalatest.wordspec.AnyWordSpecLike import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/ChainHandler.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/ChainHandler.scala index e334aab6465..f3e213350e7 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/ChainHandler.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/ChainHandler.scala @@ -1,8 +1,7 @@ package edu.uci.ics.amber.engine.architecture.control.utils import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands._ -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns._ +import edu.uci.ics.amber.engine.architecture.rpc._ trait ChainHandler { this: TesterAsyncRPCHandlerInitializer => diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/CollectHandler.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/CollectHandler.scala index 79170d89525..f21bfae563e 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/CollectHandler.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/CollectHandler.scala @@ -1,8 +1,7 @@ package edu.uci.ics.amber.engine.architecture.control.utils import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands._ -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns._ +import edu.uci.ics.amber.engine.architecture.rpc._ import scala.util.Random diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/ErrorHandler.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/ErrorHandler.scala index 52391770fdb..e4649704fc1 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/ErrorHandler.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/ErrorHandler.scala @@ -1,8 +1,7 @@ package edu.uci.ics.amber.engine.architecture.control.utils import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands._ -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns._ +import edu.uci.ics.amber.engine.architecture.rpc._ trait ErrorHandler { this: TesterAsyncRPCHandlerInitializer => diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/MultiCallHandler.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/MultiCallHandler.scala index d1b84a936e8..5fe75f7f0ec 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/MultiCallHandler.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/MultiCallHandler.scala @@ -1,9 +1,8 @@ package edu.uci.ics.amber.engine.architecture.control.utils import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands._ -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns._ -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.architecture.rpc._ +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity trait MultiCallHandler { this: TesterAsyncRPCHandlerInitializer => diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/NestedHandler.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/NestedHandler.scala index 4cb84bfe03c..d8f84514d1a 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/NestedHandler.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/NestedHandler.scala @@ -1,8 +1,7 @@ package edu.uci.ics.amber.engine.architecture.control.utils import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands._ -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns._ +import edu.uci.ics.amber.engine.architecture.rpc._ trait NestedHandler { this: TesterAsyncRPCHandlerInitializer => diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/PingPongHandler.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/PingPongHandler.scala index 1247383d781..9d390a60a08 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/PingPongHandler.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/PingPongHandler.scala @@ -1,8 +1,7 @@ package edu.uci.ics.amber.engine.architecture.control.utils import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands._ -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns._ +import edu.uci.ics.amber.engine.architecture.rpc._ trait PingPongHandler { this: TesterAsyncRPCHandlerInitializer => diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/RecursionHandler.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/RecursionHandler.scala index f482a894bbe..e2774232a57 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/RecursionHandler.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/RecursionHandler.scala @@ -1,8 +1,7 @@ package edu.uci.ics.amber.engine.architecture.control.utils import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands._ -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns._ +import edu.uci.ics.amber.engine.architecture.rpc._ trait RecursionHandler { this: TesterAsyncRPCHandlerInitializer => diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/TesterAsyncRPCHandlerInitializer.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/TesterAsyncRPCHandlerInitializer.scala index c93d6655c69..dbb47578364 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/TesterAsyncRPCHandlerInitializer.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/TesterAsyncRPCHandlerInitializer.scala @@ -2,10 +2,10 @@ package edu.uci.ics.amber.engine.architecture.control.utils import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.control.utils.TrivialControlTester.ControlTesterRPCClient -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext -import edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterFs2Grpc +import edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext +import edu.uci.ics.amber.engine.architecture.rpc.RPCTesterFs2Grpc import edu.uci.ics.amber.engine.common.rpc.{AsyncRPCHandlerInitializer, AsyncRPCServer} -import edu.uci.ics.amber.engine.common.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.common.ActorVirtualIdentity class TesterAsyncRPCHandlerInitializer( val myID: ActorVirtualIdentity, diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/TrivialControlTester.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/TrivialControlTester.scala index 6f43f56e095..231696c196b 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/TrivialControlTester.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/utils/TrivialControlTester.scala @@ -5,8 +5,8 @@ import edu.uci.ics.amber.engine.architecture.common.WorkflowActor.NetworkAck import edu.uci.ics.amber.engine.architecture.common.{AmberProcessor, WorkflowActor} import edu.uci.ics.amber.engine.architecture.control.utils.TrivialControlTester.ControlTesterRPCClient import edu.uci.ics.amber.engine.architecture.messaginglayer.NetworkOutputGateway -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.AsyncRPCContext -import edu.uci.ics.amber.engine.architecture.rpc.testerservice.RPCTesterFs2Grpc +import edu.uci.ics.amber.engine.architecture.rpc.AsyncRPCContext +import edu.uci.ics.amber.engine.architecture.rpc.RPCTesterFs2Grpc import edu.uci.ics.amber.engine.common.CheckpointState import edu.uci.ics.amber.engine.common.ambermessage.WorkflowMessage.getInMemSize import edu.uci.ics.amber.engine.common.ambermessage.{ @@ -15,7 +15,7 @@ import edu.uci.ics.amber.engine.common.ambermessage.{ WorkflowFIFOMessage } import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} object TrivialControlTester { class ControlTesterRPCClient(outputGateway: NetworkOutputGateway, actorId: ActorVirtualIdentity) diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/NetworkInputGatewaySpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/NetworkInputGatewaySpec.scala index 5dff8d7b8b1..20b3aaba46b 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/NetworkInputGatewaySpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/NetworkInputGatewaySpec.scala @@ -2,7 +2,7 @@ package edu.uci.ics.amber.engine.architecture.messaginglayer import edu.uci.ics.amber.engine.common.ambermessage.{DataFrame, WorkflowFIFOMessage} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema, TupleLike} -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} import org.scalamock.scalatest.MockFactory import org.scalatest.flatspec.AnyFlatSpec diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/OutputManagerSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/OutputManagerSpec.scala index c404f4601e2..69192c085a0 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/OutputManagerSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/OutputManagerSpec.scala @@ -1,17 +1,17 @@ package edu.uci.ics.amber.engine.architecture.messaginglayer import com.softwaremill.macwire.wire -import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning +import edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning import edu.uci.ics.amber.engine.common.ambermessage._ import edu.uci.ics.amber.engine.common.model.EndOfInputChannel import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema, TupleLike} -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.common.{ ActorVirtualIdentity, ChannelIdentity, OperatorIdentity, PhysicalOpIdentity } -import edu.uci.ics.amber.engine.common.workflow.{PhysicalLink, PortIdentity} +import edu.uci.ics.amber.engine.common.{PhysicalLink, PortIdentity} import org.scalamock.scalatest.MockFactory import org.scalatest.flatspec.AnyFlatSpec diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/RangeBasedShuffleSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/RangeBasedShuffleSpec.scala index ada159087de..128026b2157 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/RangeBasedShuffleSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/messaginglayer/RangeBasedShuffleSpec.scala @@ -1,9 +1,9 @@ package edu.uci.ics.amber.engine.architecture.messaginglayer import edu.uci.ics.amber.engine.architecture.sendsemantics.partitioners.RangeBasedShufflePartitioner -import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.RangeBasedShufflePartitioning +import edu.uci.ics.amber.engine.architecture.sendsemantics.RangeBasedShufflePartitioning import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema, Tuple} -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} import org.scalamock.scalatest.MockFactory import org.scalatest.flatspec.AnyFlatSpec diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonWorkflowWorkerSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonWorkflowWorkerSpec.scala index 617551924fd..aa1fa2ef0ca 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonWorkflowWorkerSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonWorkflowWorkerSpec.scala @@ -20,8 +20,8 @@ // WorkflowFIFOMessage //} //import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient.{ControlInvocation, ReturnInvocation} -//import edu.uci.ics.amber.engine.common.virtualidentity.util.CONTROLLER -//import edu.uci.ics.amber.engine.common.virtualidentity.{ +//import edu.uci.ics.amber.engine.common.util.CONTROLLER +//import edu.uci.ics.amber.engine.common.{ // ActorVirtualIdentity, // PhysicalLink, // PhysicalLink, diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/scheduling/ExpansionGreedyRegionPlanGeneratorSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/scheduling/ExpansionGreedyRegionPlanGeneratorSpec.scala index 515f4757212..2fb447f8a98 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/scheduling/ExpansionGreedyRegionPlanGeneratorSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/scheduling/ExpansionGreedyRegionPlanGeneratorSpec.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.engine.architecture.scheduling -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.OperatorIdentity +import edu.uci.ics.amber.engine.common.PortIdentity import edu.uci.ics.amber.engine.e2e.TestOperators import edu.uci.ics.amber.engine.e2e.TestUtils.buildWorkflow import edu.uci.ics.amber.engine.common.model.WorkflowContext diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/worker/DPThreadSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/worker/DPThreadSpec.scala index f3e66dfcd16..0bb323c297c 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/worker/DPThreadSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/worker/DPThreadSpec.scala @@ -2,8 +2,8 @@ package edu.uci.ics.amber.engine.architecture.worker import edu.uci.ics.amber.engine.architecture.logreplay.{ReplayLogManager, ReplayLogRecord} import edu.uci.ics.amber.engine.architecture.messaginglayer.WorkerTimerService -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{AsyncRPCContext, EmptyRequest} -import edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.{ +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, EmptyRequest} +import edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.{ METHOD_PAUSE_WORKER, METHOD_RESUME_WORKER } @@ -17,9 +17,9 @@ import edu.uci.ics.amber.engine.common.executor.OperatorExecutor import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema, Tuple, TupleLike} import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient.ControlInvocation import edu.uci.ics.amber.engine.common.storage.SequentialRecordStorage -import edu.uci.ics.amber.engine.common.virtualidentity.util.SELF -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.util.SELF +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.PortIdentity import org.scalamock.scalatest.MockFactory import org.scalatest.flatspec.AnyFlatSpec diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessorSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessorSpec.scala index e0f7f28bd08..5d747a267e8 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessorSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessorSpec.scala @@ -1,27 +1,27 @@ package edu.uci.ics.amber.engine.architecture.worker import edu.uci.ics.amber.engine.architecture.messaginglayer.WorkerTimerService -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{AsyncRPCContext, EmptyRequest} -import edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.{ +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, EmptyRequest} +import edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.{ METHOD_FLUSH_NETWORK_BUFFER, METHOD_OPEN_EXECUTOR } import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.MainThreadDelegateMessage -import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.READY +import edu.uci.ics.amber.engine.architecture.worker.WorkerState.READY import edu.uci.ics.amber.engine.common.VirtualIdentityUtils import edu.uci.ics.amber.engine.common.ambermessage.{DataFrame, MarkerFrame, WorkflowFIFOMessage} import edu.uci.ics.amber.engine.common.executor.OperatorExecutor import edu.uci.ics.amber.engine.common.model.EndOfInputChannel import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema, Tuple, TupleLike} import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient.ControlInvocation -import edu.uci.ics.amber.engine.common.virtualidentity.util.CONTROLLER -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.common.util.CONTROLLER +import edu.uci.ics.amber.engine.common.{ ActorVirtualIdentity, ChannelIdentity, OperatorIdentity, PhysicalOpIdentity } -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity import edu.uci.ics.amber.engine.common.model.WorkflowContext.DEFAULT_WORKFLOW_ID import org.scalamock.scalatest.MockFactory import org.scalatest.BeforeAndAfterEach diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/worker/WorkerSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/worker/WorkerSpec.scala index d198d6c6750..470eba08f9d 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/worker/WorkerSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/worker/WorkerSpec.scala @@ -8,7 +8,7 @@ import com.google.protobuf.any.{Any => ProtoAny} import edu.uci.ics.amber.clustering.SingleNodeListener import edu.uci.ics.amber.engine.architecture.common.WorkflowActor.NetworkMessage import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AddInputChannelRequest, AddPartitioningRequest, AssignPortRequest, @@ -17,7 +17,7 @@ import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ EmptyRequest, InitializeExecutorRequest } -import edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.{ +import edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.{ METHOD_ADD_INPUT_CHANNEL, METHOD_ADD_PARTITIONING, METHOD_ASSIGN_PORT, @@ -25,7 +25,7 @@ import edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc METHOD_INITIALIZE_EXECUTOR } import edu.uci.ics.amber.engine.architecture.scheduling.config.WorkerConfig -import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning +import edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.{ MainThreadDelegateMessage, WorkerReplayInitialization @@ -41,14 +41,14 @@ import edu.uci.ics.amber.engine.common.model.tuple.{ TupleLike } import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient -import edu.uci.ics.amber.engine.common.virtualidentity.util.CONTROLLER -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.common.util.CONTROLLER +import edu.uci.ics.amber.engine.common.{ ActorVirtualIdentity, ChannelIdentity, OperatorIdentity, PhysicalOpIdentity } -import edu.uci.ics.amber.engine.common.workflow.{PhysicalLink, PortIdentity} +import edu.uci.ics.amber.engine.common.{PhysicalLink, PortIdentity} import org.scalamock.scalatest.MockFactory import org.scalatest.BeforeAndAfterAll import org.scalatest.flatspec.AnyFlatSpecLike diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/e2e/BatchSizePropagationSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/e2e/BatchSizePropagationSpec.scala index 6becd96b2d3..5f53ea0edde 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/e2e/BatchSizePropagationSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/e2e/BatchSizePropagationSpec.scala @@ -5,7 +5,7 @@ import akka.testkit.{ImplicitSender, TestKit} import akka.util.Timeout import edu.uci.ics.amber.clustering.SingleNodeListener import edu.uci.ics.amber.engine.architecture.controller._ -import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.{ +import edu.uci.ics.amber.engine.architecture.sendsemantics.{ BroadcastPartitioning, HashBasedShufflePartitioning, OneToOnePartitioning, @@ -14,7 +14,7 @@ import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.{ } import edu.uci.ics.amber.engine.common.model.WorkflowContext import edu.uci.ics.amber.engine.common.model.WorkflowSettings -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity import edu.uci.ics.texera.workflow.common.storage.OpResultStorage import org.scalatest.flatspec.AnyFlatSpecLike import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/e2e/DataProcessingSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/e2e/DataProcessingSpec.scala index 7e46e3c6b72..ea32ce3da5f 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/e2e/DataProcessingSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/e2e/DataProcessingSpec.scala @@ -8,14 +8,14 @@ import ch.vorburger.mariadb4j.DB import com.twitter.util.{Await, Duration, Promise} import edu.uci.ics.amber.clustering.SingleNodeListener import edu.uci.ics.amber.engine.architecture.controller._ -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.COMPLETED +import edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest import edu.uci.ics.amber.engine.common.AmberRuntime import edu.uci.ics.amber.engine.common.client.AmberClient import edu.uci.ics.amber.engine.common.model.WorkflowContext import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Tuple} -import edu.uci.ics.amber.engine.common.virtualidentity.OperatorIdentity -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.OperatorIdentity +import edu.uci.ics.amber.engine.common.PortIdentity +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState.COMPLETED import edu.uci.ics.amber.engine.e2e.TestUtils.buildWorkflow import edu.uci.ics.texera.workflow.common.storage.OpResultStorage import edu.uci.ics.texera.workflow.common.workflow._ diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/e2e/PauseSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/e2e/PauseSpec.scala index ba9c45d1e60..487925d67a5 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/e2e/PauseSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/e2e/PauseSpec.scala @@ -8,12 +8,12 @@ import com.twitter.util.{Await, Promise} import com.typesafe.scalalogging.Logger import edu.uci.ics.amber.clustering.SingleNodeListener import edu.uci.ics.amber.engine.architecture.controller.{ControllerConfig, ExecutionStateUpdate} -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.EmptyRequest -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.COMPLETED +import edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest import edu.uci.ics.amber.engine.common.AmberRuntime import edu.uci.ics.amber.engine.common.client.AmberClient import edu.uci.ics.amber.engine.common.model.WorkflowContext -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState.COMPLETED import edu.uci.ics.texera.workflow.common.operators.LogicalOp import edu.uci.ics.texera.workflow.common.storage.OpResultStorage import edu.uci.ics.texera.workflow.common.workflow.LogicalLink diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/CheckpointSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/CheckpointSpec.scala index 11875a186a6..83d42b3b0a2 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/CheckpointSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/CheckpointSpec.scala @@ -4,23 +4,9 @@ import akka.actor.{ActorSystem, Props} import akka.serialization.SerializationExtension import com.twitter.util.{Await, Duration} import edu.uci.ics.amber.clustering.SingleNodeListener -import edu.uci.ics.amber.engine.architecture.controller.{ - ControllerConfig, - ControllerProcessor, - ExecutionStateUpdate -} -import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.{ - OpExecInitInfoWithCode, - OpExecInitInfoWithFunc -} -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ - EmptyRequest, - TakeGlobalCheckpointRequest -} -import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregatedState.{ - COMPLETED, - PAUSED -} +import edu.uci.ics.amber.engine.architecture.controller.{ControllerConfig, ControllerProcessor, ExecutionStateUpdate} +import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.{OpExecInitInfoWithCode, OpExecInitInfoWithFunc} +import edu.uci.ics.amber.engine.architecture.rpc.{EmptyRequest, TakeGlobalCheckpointRequest} import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.StateRestoreConfig import edu.uci.ics.amber.engine.architecture.worker.DataProcessor import edu.uci.ics.amber.engine.common.{AmberRuntime, CheckpointState, CheckpointSupport} @@ -28,13 +14,10 @@ import edu.uci.ics.amber.engine.common.SerializedState.{CP_STATE_KEY, DP_STATE_K import edu.uci.ics.amber.engine.common.client.AmberClient import edu.uci.ics.amber.engine.common.executor.{OperatorExecutor, SourceOperatorExecutor} import edu.uci.ics.amber.engine.common.model.WorkflowContext -import edu.uci.ics.amber.engine.common.virtualidentity.{ - ChannelMarkerIdentity, - ExecutionIdentity, - WorkflowIdentity -} -import edu.uci.ics.amber.engine.common.virtualidentity.util.{CONTROLLER, SELF} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.{ChannelMarkerIdentity, ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.engine.common.util.{CONTROLLER, SELF} +import edu.uci.ics.amber.engine.common.PortIdentity +import edu.uci.ics.amber.engine.common.WorkflowAggregatedState.{COMPLETED, PAUSED} import edu.uci.ics.amber.engine.e2e.TestOperators import edu.uci.ics.amber.engine.e2e.TestUtils.buildWorkflow import edu.uci.ics.texera.workflow.common.storage.OpResultStorage diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/LoggingSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/LoggingSpec.scala index f532dfcb52d..c05c8f613cc 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/LoggingSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/LoggingSpec.scala @@ -3,19 +3,19 @@ package edu.uci.ics.amber.engine.faulttolerance import akka.actor.ActorSystem import akka.testkit.{ImplicitSender, TestKit} import edu.uci.ics.amber.engine.architecture.logreplay.{ReplayLogManager, ReplayLogRecord} -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ +import edu.uci.ics.amber.engine.architecture.rpc.{ AddPartitioningRequest, AsyncRPCContext, EmptyRequest } -import edu.uci.ics.amber.engine.architecture.rpc.controllerservice.ControllerServiceGrpc.METHOD_WORKER_EXECUTION_COMPLETED -import edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.{ +import edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_WORKER_EXECUTION_COMPLETED +import edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.{ METHOD_ADD_PARTITIONING, METHOD_PAUSE_WORKER, METHOD_RESUME_WORKER, METHOD_START_WORKER } -import edu.uci.ics.amber.engine.architecture.sendsemantics.partitionings.OneToOnePartitioning +import edu.uci.ics.amber.engine.architecture.sendsemantics.OneToOnePartitioning import edu.uci.ics.amber.engine.common.ambermessage.{ DataFrame, WorkflowFIFOMessage, @@ -24,14 +24,14 @@ import edu.uci.ics.amber.engine.common.ambermessage.{ import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema, TupleLike} import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient.ControlInvocation import edu.uci.ics.amber.engine.common.storage.SequentialRecordStorage -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.common.{ ActorVirtualIdentity, ChannelIdentity, OperatorIdentity, PhysicalOpIdentity } -import edu.uci.ics.amber.engine.common.virtualidentity.util.{CONTROLLER, SELF} -import edu.uci.ics.amber.engine.common.workflow.{PhysicalLink, PortIdentity} +import edu.uci.ics.amber.engine.common.util.{CONTROLLER, SELF} +import edu.uci.ics.amber.engine.common.{PhysicalLink, PortIdentity} import org.scalatest.BeforeAndAfterAll import org.scalatest.flatspec.AnyFlatSpecLike diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/ReplaySpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/ReplaySpec.scala index 062ad255bc0..6c941590328 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/ReplaySpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/ReplaySpec.scala @@ -10,13 +10,13 @@ import edu.uci.ics.amber.engine.architecture.logreplay.{ ReplayOrderEnforcer } import edu.uci.ics.amber.engine.architecture.messaginglayer.NetworkInputGateway -import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{AsyncRPCContext, EmptyRequest} -import edu.uci.ics.amber.engine.architecture.rpc.workerservice.WorkerServiceGrpc.METHOD_START_WORKER +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, EmptyRequest} +import edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_START_WORKER import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessage import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient.ControlInvocation import edu.uci.ics.amber.engine.common.storage.SequentialRecordStorage -import edu.uci.ics.amber.engine.common.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} -import edu.uci.ics.amber.engine.common.virtualidentity.util.CONTROLLER +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} +import edu.uci.ics.amber.engine.common.util.CONTROLLER import org.scalatest.BeforeAndAfterAll import org.scalatest.flatspec.AnyFlatSpecLike diff --git a/core/amber/src/test/scala/edu/uci/ics/texera/workflow/common/workflow/SchemaPropagationSpec.scala b/core/amber/src/test/scala/edu/uci/ics/texera/workflow/common/workflow/SchemaPropagationSpec.scala index 9e800a789e9..74ef5053045 100644 --- a/core/amber/src/test/scala/edu/uci/ics/texera/workflow/common/workflow/SchemaPropagationSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/texera/workflow/common/workflow/SchemaPropagationSpec.scala @@ -2,12 +2,12 @@ package edu.uci.ics.texera.workflow.common.workflow import edu.uci.ics.amber.engine.common.model.{PhysicalOp, WorkflowContext} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.virtualidentity.{ +import edu.uci.ics.amber.engine.common.{ ExecutionIdentity, OperatorIdentity, WorkflowIdentity } -import edu.uci.ics.amber.engine.common.workflow.{InputPort, OutputPort, PortIdentity} +import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.OperatorInfo import edu.uci.ics.texera.workflow.common.operators.LogicalOp import edu.uci.ics.texera.workflow.common.operators.source.SourceOperatorDescriptor diff --git a/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/intersect/IntersectOpExecSpec.scala b/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/intersect/IntersectOpExecSpec.scala index 5ee876d2823..06f8030d802 100644 --- a/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/intersect/IntersectOpExecSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/intersect/IntersectOpExecSpec.scala @@ -7,8 +7,8 @@ import edu.uci.ics.amber.engine.common.model.tuple.{ Tuple, TupleLike } -import edu.uci.ics.amber.engine.common.virtualidentity.{OperatorIdentity, PhysicalOpIdentity} -import edu.uci.ics.amber.engine.common.workflow.{PhysicalLink, PortIdentity} +import edu.uci.ics.amber.engine.common.{OperatorIdentity, PhysicalOpIdentity} +import edu.uci.ics.amber.engine.common.{PhysicalLink, PortIdentity} import org.scalatest.BeforeAndAfter import org.scalatest.flatspec.AnyFlatSpec diff --git a/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/intervalJoin/IntervalOpExecSpec.scala b/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/intervalJoin/IntervalOpExecSpec.scala index afd97287758..05c590b284b 100644 --- a/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/intervalJoin/IntervalOpExecSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/intervalJoin/IntervalOpExecSpec.scala @@ -8,8 +8,8 @@ import edu.uci.ics.amber.engine.common.model.tuple.{ Tuple, TupleLike } -import edu.uci.ics.amber.engine.common.virtualidentity.{OperatorIdentity, PhysicalOpIdentity} -import edu.uci.ics.amber.engine.common.workflow.{PhysicalLink, PortIdentity} +import edu.uci.ics.amber.engine.common.{OperatorIdentity, PhysicalOpIdentity} +import edu.uci.ics.amber.engine.common.{PhysicalLink, PortIdentity} import org.scalatest.BeforeAndAfter import org.scalatest.flatspec.AnyFlatSpec diff --git a/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/CSVScanSourceOpDescSpec.scala b/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/CSVScanSourceOpDescSpec.scala index 858b2188309..54b9f39e8b8 100644 --- a/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/CSVScanSourceOpDescSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/source/scan/csv/CSVScanSourceOpDescSpec.scala @@ -2,7 +2,7 @@ package edu.uci.ics.texera.workflow.operators.source.scan.csv import edu.uci.ics.amber.engine.common.model.WorkflowContext import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity import WorkflowContext.{DEFAULT_EXECUTION_ID, DEFAULT_WORKFLOW_ID} import edu.uci.ics.texera.workflow.common.storage.FileResolver import org.scalatest.BeforeAndAfter diff --git a/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/unneststring/UnnestStringOpExecSpec.scala b/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/unneststring/UnnestStringOpExecSpec.scala index b812ae9d36a..824bf99c7ec 100644 --- a/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/unneststring/UnnestStringOpExecSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/texera/workflow/operators/unneststring/UnnestStringOpExecSpec.scala @@ -7,7 +7,7 @@ import edu.uci.ics.amber.engine.common.model.tuple.{ SchemaEnforceable, Tuple } -import edu.uci.ics.amber.engine.common.workflow.PortIdentity +import edu.uci.ics.amber.engine.common.PortIdentity import org.scalatest.BeforeAndAfter import org.scalatest.flatspec.AnyFlatSpec From 88553ced86fff9d5d5897e504a59b06d58140b75 Mon Sep 17 00:00:00 2001 From: Shengquan Ni <13672781+shengquan-ni@users.noreply.github.com> Date: Thu, 31 Oct 2024 17:36:07 -0700 Subject: [PATCH 2/2] format --- .../actorcommand/backpressure_handler.py | 2 +- .../core/architecture/rpc/async_rpc_client.py | 2 +- .../core/architecture/rpc/async_rpc_server.py | 6 +-- .../main/python/core/models/internal_queue.py | 3 +- .../main/python/core/runnables/main_loop.py | 2 +- .../python/core/runnables/network_receiver.py | 6 ++- .../python/core/runnables/network_sender.py | 8 ++-- .../core/runnables/test_console_message.py | 6 ++- .../python/core/runnables/test_main_loop.py | 2 +- .../core/runnables/test_network_receiver.py | 2 +- .../engine/architecture/python/__init__.py | 38 +++++++++++++++++ .../uci/ics/amber/engine/common/__init__.py | 41 ++++++------------- .../architecture/controller/ClientEvent.scala | 6 ++- .../architecture/controller/Controller.scala | 5 +-- .../controller/ControllerTimerService.scala | 5 +-- .../controller/execution/ExecutionUtils.scala | 6 ++- .../execution/OperatorExecution.scala | 8 +++- .../execution/RegionExecution.scala | 7 +++- .../execution/WorkflowExecution.scala | 6 ++- .../ChannelMarkerHandler.scala | 5 +-- .../ConsoleMessageHandler.scala | 5 +-- .../promisehandlers/DebugCommandHandler.scala | 5 +-- .../EvaluatePythonExpressionHandler.scala | 5 +-- .../promisehandlers/LinkWorkersHandler.scala | 8 +++- .../RetrieveWorkflowStateHandler.scala | 5 +-- .../WorkerStateUpdatedHandler.scala | 5 +-- .../pythonworker/PythonProxyClient.scala | 18 ++++++-- .../pythonworker/PythonProxyServer.scala | 5 ++- .../WorkerBatchInternalQueue.scala | 6 +-- .../RegionExecutionCoordinator.scala | 18 ++++++-- .../worker/ChannelMarkerManager.scala | 5 +-- .../architecture/worker/DataProcessor.scala | 6 +-- .../worker/managers/StatisticsManager.scala | 5 +-- .../AddInputChannelHandler.scala | 5 +-- .../AddPartitioningHandler.scala | 5 +-- .../promisehandlers/AssignPortHandler.scala | 5 +-- .../FinalizeCheckpointHandler.scala | 5 +-- .../InitializeExecutorHandler.scala | 5 +-- .../PrepareCheckpointHandler.scala | 5 +-- .../engine/common/client/ClientActor.scala | 6 +-- .../engine/common/model/PhysicalPlan.scala | 6 +-- .../ics/texera/web/TexeraWebApplication.scala | 30 +++++++++++--- .../texera/web/WorkflowLifecycleManager.scala | 6 ++- .../resource/WorkflowWebsocketResource.scala | 6 ++- .../service/EmailNotificationService.scala | 1 - .../web/service/ExecutionConsoleService.scala | 6 +-- .../web/service/ExecutionResultService.scala | 13 +++++- .../web/service/ExecutionRuntimeService.scala | 8 +++- .../web/service/ExecutionStatsService.scala | 24 +++++++++-- .../FriesReconfigurationAlgorithm.scala | 5 +-- .../service/WorkflowExecutionService.scala | 6 ++- .../texera/web/service/WorkflowService.scala | 5 ++- .../web/storage/ExecutionStateStore.scala | 8 +++- .../workflow/common/operators/LogicalOp.scala | 6 +-- .../operators/aggregate/AggregateOpDesc.scala | 6 +-- .../operators/hashJoin/HashJoinOpDesc.scala | 6 +-- .../source/cache/CacheSourceOpDesc.scala | 6 +-- .../control/TrivialControlSpec.scala | 6 +-- .../faulttolerance/CheckpointSpec.scala | 11 ++++- .../workflow/SchemaPropagationSpec.scala | 6 +-- 60 files changed, 270 insertions(+), 199 deletions(-) create mode 100644 core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/python/__init__.py diff --git a/core/amber/src/main/python/core/architecture/handlers/actorcommand/backpressure_handler.py b/core/amber/src/main/python/core/architecture/handlers/actorcommand/backpressure_handler.py index e1171cf8870..240a0a9cae0 100644 --- a/core/amber/src/main/python/core/architecture/handlers/actorcommand/backpressure_handler.py +++ b/core/amber/src/main/python/core/architecture/handlers/actorcommand/backpressure_handler.py @@ -4,6 +4,7 @@ from core.models.internal_queue import ControlElement, InternalQueue from core.util import set_one_of +from proto.edu.uci.ics.amber.engine.architecture.python import ControlPayloadV2 from proto.edu.uci.ics.amber.engine.architecture.rpc import ( ControlInvocation, ControlRequest, @@ -14,7 +15,6 @@ from proto.edu.uci.ics.amber.engine.common import ( Backpressure, ActorVirtualIdentity, - ControlPayloadV2, ) diff --git a/core/amber/src/main/python/core/architecture/rpc/async_rpc_client.py b/core/amber/src/main/python/core/architecture/rpc/async_rpc_client.py index 6bd7c8a9cfb..a31cec7a2a9 100644 --- a/core/amber/src/main/python/core/architecture/rpc/async_rpc_client.py +++ b/core/amber/src/main/python/core/architecture/rpc/async_rpc_client.py @@ -9,6 +9,7 @@ from core.architecture.managers.context import Context from core.models.internal_queue import InternalQueue, ControlElement from core.util import set_one_of +from proto.edu.uci.ics.amber.engine.architecture.python import ControlPayloadV2 from proto.edu.uci.ics.amber.engine.architecture.rpc import ( AsyncRpcContext, ReturnInvocation, @@ -20,7 +21,6 @@ ) from proto.edu.uci.ics.amber.engine.common import ( ActorVirtualIdentity, - ControlPayloadV2, ) R = TypeVar("R") diff --git a/core/amber/src/main/python/core/architecture/rpc/async_rpc_server.py b/core/amber/src/main/python/core/architecture/rpc/async_rpc_server.py index b214a0d5f2d..38eff168601 100644 --- a/core/amber/src/main/python/core/architecture/rpc/async_rpc_server.py +++ b/core/amber/src/main/python/core/architecture/rpc/async_rpc_server.py @@ -7,17 +7,17 @@ ) from core.models.internal_queue import InternalQueue, ControlElement from core.util import get_one_of, set_one_of +from proto.edu.uci.ics.amber.engine.architecture.python import ControlPayloadV2 from proto.edu.uci.ics.amber.engine.architecture.rpc import ( ReturnInvocation, ControlRequest, ControlInvocation, ControlReturn, - ControlError, + ControlException, ErrorLanguage, ) from proto.edu.uci.ics.amber.engine.common import ( ActorVirtualIdentity, - ControlPayloadV2, ) @@ -89,7 +89,7 @@ def receive( # Construct a ControlError message in case of an exception. control_return: ControlReturn = set_one_of( ControlReturn, - ControlError( + ControlException( error_message=str(exception), language=ErrorLanguage.PYTHON ), ) diff --git a/core/amber/src/main/python/core/models/internal_queue.py b/core/amber/src/main/python/core/models/internal_queue.py index 36e271983e8..920f02a75b5 100644 --- a/core/amber/src/main/python/core/models/internal_queue.py +++ b/core/amber/src/main/python/core/models/internal_queue.py @@ -11,7 +11,8 @@ LinkedBlockingMultiQueue, ) from core.util.customized_queue.queue_base import IQueue, QueueElement -from proto.edu.uci.ics.amber.engine.common import ActorVirtualIdentity, ControlPayloadV2 +from proto.edu.uci.ics.amber.engine.architecture.python import ControlPayloadV2 +from proto.edu.uci.ics.amber.engine.common import ActorVirtualIdentity @dataclass diff --git a/core/amber/src/main/python/core/runnables/main_loop.py b/core/amber/src/main/python/core/runnables/main_loop.py index 845afe0d21a..4d84cebba26 100644 --- a/core/amber/src/main/python/core/runnables/main_loop.py +++ b/core/amber/src/main/python/core/runnables/main_loop.py @@ -27,6 +27,7 @@ from core.util import StoppableQueueBlockingRunnable, get_one_of from core.util.console_message.timestamp import current_time_in_local_timezone from core.util.customized_queue.queue_base import QueueElement +from proto.edu.uci.ics.amber.engine.architecture.python import ControlPayloadV2 from proto.edu.uci.ics.amber.engine.architecture.rpc import ( ConsoleMessage, ControlInvocation, @@ -41,7 +42,6 @@ ) from proto.edu.uci.ics.amber.engine.common import ( ActorVirtualIdentity, - ControlPayloadV2, PortIdentity, ) diff --git a/core/amber/src/main/python/core/runnables/network_receiver.py b/core/amber/src/main/python/core/runnables/network_receiver.py index 22d0911b9f7..1f1e3eef007 100644 --- a/core/amber/src/main/python/core/runnables/network_receiver.py +++ b/core/amber/src/main/python/core/runnables/network_receiver.py @@ -22,10 +22,12 @@ from core.proxy import ProxyServer from core.util import Stoppable, get_one_of from core.util.runnable.runnable import Runnable -from proto.edu.uci.ics.amber.engine.common import ( - PythonControlMessage, +from proto.edu.uci.ics.amber.engine.architecture.python import ( PythonDataHeader, + PythonControlMessage, PythonActorMessage, +) +from proto.edu.uci.ics.amber.engine.common import ( ActorCommand, ) diff --git a/core/amber/src/main/python/core/runnables/network_sender.py b/core/amber/src/main/python/core/runnables/network_sender.py index 031f2783902..ddf977d0750 100644 --- a/core/amber/src/main/python/core/runnables/network_sender.py +++ b/core/amber/src/main/python/core/runnables/network_sender.py @@ -8,11 +8,13 @@ from core.models.internal_queue import InternalQueueElement, DataElement, ControlElement from core.proxy import ProxyClient from core.util import StoppableQueueBlockingRunnable +from proto.edu.uci.ics.amber.engine.architecture.python import ( + PythonDataHeader, + PythonControlMessage, + ControlPayloadV2, +) from proto.edu.uci.ics.amber.engine.common import ( ActorVirtualIdentity, - ControlPayloadV2, - PythonControlMessage, - PythonDataHeader, ) diff --git a/core/amber/src/main/python/core/runnables/test_console_message.py b/core/amber/src/main/python/core/runnables/test_console_message.py index a643a789855..e93dd2efd06 100644 --- a/core/amber/src/main/python/core/runnables/test_console_message.py +++ b/core/amber/src/main/python/core/runnables/test_console_message.py @@ -3,6 +3,10 @@ from core.models.internal_queue import InternalQueue from core.util.buffer.timed_buffer import TimedBuffer from core.util import set_one_of +from proto.edu.uci.ics.amber.engine.architecture.python import ( + ControlPayloadV2, + PythonControlMessage, +) from proto.edu.uci.ics.amber.engine.architecture.rpc import ( ControlInvocation, ControlRequest, @@ -11,8 +15,6 @@ ) from proto.edu.uci.ics.amber.engine.common import ( ActorVirtualIdentity, - ControlPayloadV2, - PythonControlMessage, ) diff --git a/core/amber/src/main/python/core/runnables/test_main_loop.py b/core/amber/src/main/python/core/runnables/test_main_loop.py index 05cfdf9362b..2ac26432e13 100644 --- a/core/amber/src/main/python/core/runnables/test_main_loop.py +++ b/core/amber/src/main/python/core/runnables/test_main_loop.py @@ -16,6 +16,7 @@ from core.models.marker import EndOfInputChannel from core.runnables import MainLoop from core.util import set_one_of +from proto.edu.uci.ics.amber.engine.architecture.python import ControlPayloadV2 from proto.edu.uci.ics.amber.engine.architecture.rpc import ( ControlRequest, AssignPortRequest, @@ -44,7 +45,6 @@ ) from proto.edu.uci.ics.amber.engine.common import ( ActorVirtualIdentity, - ControlPayloadV2, PhysicalLink, PhysicalOpIdentity, OperatorIdentity, diff --git a/core/amber/src/main/python/core/runnables/test_network_receiver.py b/core/amber/src/main/python/core/runnables/test_network_receiver.py index f2ca1d640c8..8ea7ef93a47 100644 --- a/core/amber/src/main/python/core/runnables/test_network_receiver.py +++ b/core/amber/src/main/python/core/runnables/test_network_receiver.py @@ -10,10 +10,10 @@ from core.runnables.network_receiver import NetworkReceiver from core.runnables.network_sender import NetworkSender from core.util.proto import set_one_of +from proto.edu.uci.ics.amber.engine.architecture.python import ControlPayloadV2 from proto.edu.uci.ics.amber.engine.architecture.rpc import ControlInvocation from proto.edu.uci.ics.amber.engine.common import ( ActorVirtualIdentity, - ControlPayloadV2, ) diff --git a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/python/__init__.py b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/python/__init__.py new file mode 100644 index 00000000000..c335dfc7007 --- /dev/null +++ b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/python/__init__.py @@ -0,0 +1,38 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# sources: edu/uci/ics/amber/engine/architecture/python/proxy_message.proto +# plugin: python-betterproto +# This file has been @generated + +from dataclasses import dataclass + +import betterproto + +from ... import common as __common__ +from .. import rpc as _rpc__ + + +@dataclass(eq=False, repr=False) +class ControlPayloadV2(betterproto.Message): + control_invocation: "_rpc__.ControlInvocation" = betterproto.message_field( + 1, group="value" + ) + return_invocation: "_rpc__.ReturnInvocation" = betterproto.message_field( + 2, group="value" + ) + + +@dataclass(eq=False, repr=False) +class PythonDataHeader(betterproto.Message): + tag: "__common__.ActorVirtualIdentity" = betterproto.message_field(1) + payload_type: str = betterproto.string_field(2) + + +@dataclass(eq=False, repr=False) +class PythonControlMessage(betterproto.Message): + tag: "__common__.ActorVirtualIdentity" = betterproto.message_field(1) + payload: "ControlPayloadV2" = betterproto.message_field(2) + + +@dataclass(eq=False, repr=False) +class PythonActorMessage(betterproto.Message): + payload: "__common__.ActorCommand" = betterproto.message_field(1) diff --git a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/common/__init__.py b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/common/__init__.py index 95781507601..96e484fe807 100644 --- a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/common/__init__.py +++ b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/common/__init__.py @@ -1,5 +1,5 @@ # Generated by the protocol buffer compiler. DO NOT EDIT! -# sources: edu/uci/ics/amber/engine/common/actor_message.proto, edu/uci/ics/amber/engine/common/amber_message.proto, edu/uci/ics/amber/engine/common/virtual_identity.proto, edu/uci/ics/amber/engine/common/workflow.proto, edu/uci/ics/amber/engine/common/workflow_metrics.proto, edu/uci/ics/amber/engine/common/workflow_runtime_state.proto +# sources: edu/uci/ics/amber/engine/common/actor_message.proto, edu/uci/ics/amber/engine/common/virtual_identity.proto, edu/uci/ics/amber/engine/common/workflow.proto, edu/uci/ics/amber/engine/common/workflow_metrics.proto, edu/uci/ics/amber/engine/common/workflow_runtime_state.proto # plugin: python-betterproto # This file has been @generated @@ -123,28 +123,6 @@ class OperatorMetrics(betterproto.Message): operator_statistics: "OperatorStatistics" = betterproto.message_field(2) -@dataclass(eq=False, repr=False) -class ControlPayloadV2(betterproto.Message): - control_invocation: "_architecture_rpc__.ControlInvocation" = ( - betterproto.message_field(1, group="value") - ) - return_invocation: "_architecture_rpc__.ReturnInvocation" = ( - betterproto.message_field(2, group="value") - ) - - -@dataclass(eq=False, repr=False) -class PythonDataHeader(betterproto.Message): - tag: "ActorVirtualIdentity" = betterproto.message_field(1) - payload_type: str = betterproto.string_field(2) - - -@dataclass(eq=False, repr=False) -class PythonControlMessage(betterproto.Message): - tag: "ActorVirtualIdentity" = betterproto.message_field(1) - payload: "ControlPayloadV2" = betterproto.message_field(2) - - @dataclass(eq=False, repr=False) class Backpressure(betterproto.Message): enable_backpressure: bool = betterproto.bool_field(1) @@ -161,11 +139,6 @@ class ActorCommand(betterproto.Message): credit_update: "CreditUpdate" = betterproto.message_field(2, group="sealed_value") -@dataclass(eq=False, repr=False) -class PythonActorMessage(betterproto.Message): - payload: "ActorCommand" = betterproto.message_field(1) - - @dataclass(eq=False, repr=False) class BreakpointFault(betterproto.Message): worker_name: str = betterproto.string_field(1) @@ -219,6 +192,18 @@ class OperatorWorkerMapping(betterproto.Message): worker_ids: List[str] = betterproto.string_field(2) +@dataclass(eq=False, repr=False) +class ExecutionStatsStore(betterproto.Message): + start_timestamp: int = betterproto.int64_field(1) + end_timestamp: int = betterproto.int64_field(2) + operator_info: Dict[str, "OperatorMetrics"] = betterproto.map_field( + 3, betterproto.TYPE_STRING, betterproto.TYPE_MESSAGE + ) + operator_worker_mapping: List["OperatorWorkerMapping"] = betterproto.message_field( + 4 + ) + + @dataclass(eq=False, repr=False) class WorkflowFatalError(betterproto.Message): type: "FatalErrorType" = betterproto.enum_field(1) diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ClientEvent.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ClientEvent.scala index 44541550456..1f1bc204eda 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ClientEvent.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ClientEvent.scala @@ -2,7 +2,11 @@ package edu.uci.ics.amber.engine.architecture.controller import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessagePayload import edu.uci.ics.amber.engine.common.model.tuple.Tuple -import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, OperatorMetrics, WorkflowAggregatedState} +import edu.uci.ics.amber.engine.common.{ + ActorVirtualIdentity, + OperatorMetrics, + WorkflowAggregatedState +} trait ClientEvent extends WorkflowFIFOMessagePayload diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/Controller.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/Controller.scala index 5a43a581854..43af57d6cb6 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/Controller.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/Controller.scala @@ -9,10 +9,7 @@ import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.{ FaultToleranceConfig, StateRestoreConfig } -import edu.uci.ics.amber.engine.architecture.rpc.{ - ChannelMarkerPayload, - ControlInvocation -} +import edu.uci.ics.amber.engine.architecture.rpc.{ChannelMarkerPayload, ControlInvocation} import edu.uci.ics.amber.engine.common.ambermessage.WorkflowMessage.getInMemSize import edu.uci.ics.amber.engine.common.model.{PhysicalPlan, WorkflowContext} import edu.uci.ics.amber.engine.common.ambermessage.{ControlPayload, WorkflowFIFOMessage} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerTimerService.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerTimerService.scala index 2e64bb2e4f6..4a84fd3f32b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerTimerService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ControllerTimerService.scala @@ -2,10 +2,7 @@ package edu.uci.ics.amber.engine.architecture.controller import akka.actor.Cancellable import edu.uci.ics.amber.engine.architecture.common.AkkaActorService -import edu.uci.ics.amber.engine.architecture.rpc.{ - AsyncRPCContext, - QueryStatisticsRequest -} +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, QueryStatisticsRequest} import edu.uci.ics.amber.engine.architecture.rpc.ControllerServiceGrpc.METHOD_CONTROLLER_INITIATE_QUERY_STATISTICS import edu.uci.ics.amber.engine.common.rpc.AsyncRPCClient.ControlInvocation import edu.uci.ics.amber.engine.common.util.SELF diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/ExecutionUtils.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/ExecutionUtils.scala index 6277ff4721d..e668f8ff0ce 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/ExecutionUtils.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/ExecutionUtils.scala @@ -1,7 +1,11 @@ package edu.uci.ics.amber.engine.architecture.controller.execution import edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping -import edu.uci.ics.amber.engine.common.{OperatorMetrics, OperatorStatistics, WorkflowAggregatedState} +import edu.uci.ics.amber.engine.common.{ + OperatorMetrics, + OperatorStatistics, + WorkflowAggregatedState +} object ExecutionUtils { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/OperatorExecution.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/OperatorExecution.scala index d0b00e7a3f8..ade7e458081 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/OperatorExecution.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/OperatorExecution.scala @@ -3,7 +3,13 @@ package edu.uci.ics.amber.engine.architecture.controller.execution import edu.uci.ics.amber.engine.architecture.controller.execution.ExecutionUtils.aggregateStates import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.WorkerExecution import edu.uci.ics.amber.engine.architecture.worker.{PortTupleCountMapping, WorkerState} -import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, OperatorMetrics, OperatorStatistics, PortIdentity, WorkflowAggregatedState} +import edu.uci.ics.amber.engine.common.{ + ActorVirtualIdentity, + OperatorMetrics, + OperatorStatistics, + PortIdentity, + WorkflowAggregatedState +} import java.util import scala.jdk.CollectionConverters._ diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/RegionExecution.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/RegionExecution.scala index 3d5f2791335..700c479dbe6 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/RegionExecution.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/RegionExecution.scala @@ -3,7 +3,12 @@ package edu.uci.ics.amber.engine.architecture.controller.execution import com.rits.cloning.Cloner import edu.uci.ics.amber.engine.architecture.scheduling.Region import edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics -import edu.uci.ics.amber.engine.common.{OperatorMetrics, PhysicalLink, PhysicalOpIdentity, WorkflowAggregatedState} +import edu.uci.ics.amber.engine.common.{ + OperatorMetrics, + PhysicalLink, + PhysicalOpIdentity, + WorkflowAggregatedState +} import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/WorkflowExecution.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/WorkflowExecution.scala index c6315334c20..7add8961875 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/WorkflowExecution.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/WorkflowExecution.scala @@ -3,7 +3,11 @@ package edu.uci.ics.amber.engine.architecture.controller.execution import edu.uci.ics.amber.engine.architecture.controller.execution.ExecutionUtils.aggregateMetrics import edu.uci.ics.amber.engine.architecture.scheduling.{Region, RegionIdentity} import edu.uci.ics.amber.engine.common.WorkflowAggregatedState._ -import edu.uci.ics.amber.engine.common.{OperatorMetrics, PhysicalOpIdentity, WorkflowAggregatedState} +import edu.uci.ics.amber.engine.common.{ + OperatorMetrics, + PhysicalOpIdentity, + WorkflowAggregatedState +} import scala.collection.mutable diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ChannelMarkerHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ChannelMarkerHandler.scala index 9a1d315ee41..75f8d75390b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ChannelMarkerHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ChannelMarkerHandler.scala @@ -7,10 +7,7 @@ import edu.uci.ics.amber.engine.architecture.rpc.{ ControlInvocation, PropagateChannelMarkerRequest } -import edu.uci.ics.amber.engine.architecture.rpc.{ - ControlReturn, - PropagateChannelMarkerResponse -} +import edu.uci.ics.amber.engine.architecture.rpc.{ControlReturn, PropagateChannelMarkerResponse} import edu.uci.ics.amber.engine.common.VirtualIdentityUtils import edu.uci.ics.amber.engine.common.util.CONTROLLER import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, ChannelIdentity} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ConsoleMessageHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ConsoleMessageHandler.scala index f6155b1ba27..eeacb61d61b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ConsoleMessageHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ConsoleMessageHandler.scala @@ -2,10 +2,7 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.{ - AsyncRPCContext, - ConsoleMessageTriggeredRequest -} +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, ConsoleMessageTriggeredRequest} import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn trait ConsoleMessageHandler { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/DebugCommandHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/DebugCommandHandler.scala index 0a61ab03a4b..5561a7923a3 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/DebugCommandHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/DebugCommandHandler.scala @@ -2,10 +2,7 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.{ - AsyncRPCContext, - DebugCommandRequest -} +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, DebugCommandRequest} import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.common.ActorVirtualIdentity diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/EvaluatePythonExpressionHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/EvaluatePythonExpressionHandler.scala index 64bea378d9c..cfc078d82b7 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/EvaluatePythonExpressionHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/EvaluatePythonExpressionHandler.scala @@ -2,10 +2,7 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.{ - AsyncRPCContext, - EvaluatePythonExpressionRequest -} +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, EvaluatePythonExpressionRequest} import edu.uci.ics.amber.engine.architecture.rpc.EvaluatePythonExpressionResponse import edu.uci.ics.amber.engine.common.OperatorIdentity diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/LinkWorkersHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/LinkWorkersHandler.scala index 07049ff7f54..fd74784d420 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/LinkWorkersHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/LinkWorkersHandler.scala @@ -2,7 +2,13 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.controller.ControllerAsyncRPCHandlerInitializer -import edu.uci.ics.amber.engine.architecture.rpc.{AddInputChannelRequest, AddPartitioningRequest, AsyncRPCContext, EmptyReturn, LinkWorkersRequest} +import edu.uci.ics.amber.engine.architecture.rpc.{ + AddInputChannelRequest, + AddPartitioningRequest, + AsyncRPCContext, + EmptyReturn, + LinkWorkersRequest +} /** add a data transfer partitioning to the sender workers and update input linking * for the receiver workers of a link strategy. diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/RetrieveWorkflowStateHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/RetrieveWorkflowStateHandler.scala index 62456a2d41a..1c7101af939 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/RetrieveWorkflowStateHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/RetrieveWorkflowStateHandler.scala @@ -8,10 +8,7 @@ import edu.uci.ics.amber.engine.architecture.rpc.{ EmptyRequest, PropagateChannelMarkerRequest } -import edu.uci.ics.amber.engine.architecture.rpc.{ - RetrieveWorkflowStateResponse, - StringResponse -} +import edu.uci.ics.amber.engine.architecture.rpc.{RetrieveWorkflowStateResponse, StringResponse} import edu.uci.ics.amber.engine.architecture.rpc.WorkerServiceGrpc.METHOD_RETRIEVE_STATE import edu.uci.ics.amber.engine.common.util.SELF import edu.uci.ics.amber.engine.common.ChannelMarkerIdentity diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerStateUpdatedHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerStateUpdatedHandler.scala index 206c88f5b42..f33303445cd 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerStateUpdatedHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerStateUpdatedHandler.scala @@ -5,10 +5,7 @@ import edu.uci.ics.amber.engine.architecture.controller.{ ControllerAsyncRPCHandlerInitializer, ExecutionStatsUpdate } -import edu.uci.ics.amber.engine.architecture.rpc.{ - AsyncRPCContext, - WorkerStateUpdatedRequest -} +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, WorkerStateUpdatedRequest} import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.common.VirtualIdentityUtils diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyClient.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyClient.scala index ce5cbd4c55e..577d200aaef 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyClient.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyClient.scala @@ -3,15 +3,27 @@ package edu.uci.ics.amber.engine.architecture.pythonworker import com.google.protobuf.ByteString import com.google.protobuf.any.Any import com.twitter.util.{Await, Promise} -import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.{OpExecInitInfo, OpExecInitInfoWithCode} +import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.{ + OpExecInitInfo, + OpExecInitInfoWithCode +} import edu.uci.ics.amber.engine.architecture.python._ -import edu.uci.ics.amber.engine.architecture.pythonworker.WorkerBatchInternalQueue.{ActorCommandElement, ControlElement, DataElement} +import edu.uci.ics.amber.engine.architecture.pythonworker.WorkerBatchInternalQueue.{ + ActorCommandElement, + ControlElement, + DataElement +} import edu.uci.ics.amber.engine.architecture.rpc.{ControlInvocation, InitializeExecutorRequest} import edu.uci.ics.amber.engine.architecture.rpc.ReturnInvocation import edu.uci.ics.amber.engine.common.{AmberLogging, AmberRuntime} import edu.uci.ics.amber.engine.common.ActorCommand import edu.uci.ics.amber.engine.common.amberexception.WorkflowRuntimeException -import edu.uci.ics.amber.engine.common.ambermessage.{ControlPayload, DataFrame, DataPayload, MarkerFrame} +import edu.uci.ics.amber.engine.common.ambermessage.{ + ControlPayload, + DataFrame, + DataPayload, + MarkerFrame +} import edu.uci.ics.amber.engine.common.model.State import edu.uci.ics.amber.engine.common.model.tuple.{Schema, Tuple} import edu.uci.ics.amber.engine.common.ActorVirtualIdentity diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyServer.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyServer.scala index 0085d69806f..e3b2ce7a810 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyServer.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/PythonProxyServer.scala @@ -14,7 +14,10 @@ import java.net.ServerSocket import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable import com.twitter.util.Promise -import edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.{ControlInvocation => ControlInvocationV2, ReturnInvocation => ReturnInvocationV2} +import edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2.Value.{ + ControlInvocation => ControlInvocationV2, + ReturnInvocation => ReturnInvocationV2 +} import edu.uci.ics.amber.engine.architecture.python.{PythonControlMessage, PythonDataHeader} import edu.uci.ics.amber.engine.common.model.{EndOfInputChannel, StartOfInputChannel, State} import edu.uci.ics.amber.engine.common.model.tuple.Tuple diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/WorkerBatchInternalQueue.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/WorkerBatchInternalQueue.scala index 14da0e36af3..c56e25ae6e4 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/WorkerBatchInternalQueue.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/pythonworker/WorkerBatchInternalQueue.scala @@ -3,11 +3,7 @@ package edu.uci.ics.amber.engine.architecture.pythonworker import edu.uci.ics.amber.engine.architecture.python.ControlPayloadV2 import edu.uci.ics.amber.engine.architecture.pythonworker.WorkerBatchInternalQueue._ import edu.uci.ics.amber.engine.common.{ActorCommand} -import edu.uci.ics.amber.engine.common.ambermessage.{ - ControlPayload, - DataFrame, - DataPayload -} +import edu.uci.ics.amber.engine.common.ambermessage.{ControlPayload, DataFrame, DataPayload} import edu.uci.ics.amber.engine.common.ChannelIdentity import lbmq.LinkedBlockingMultiQueue diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala index 914bbf23839..e9e60403423 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala @@ -4,9 +4,21 @@ import com.google.protobuf.any.Any import com.google.protobuf.ByteString import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.common.AkkaActorService -import edu.uci.ics.amber.engine.architecture.controller.{ControllerConfig, ExecutionStatsUpdate, WorkerAssignmentUpdate} -import edu.uci.ics.amber.engine.architecture.controller.execution.{OperatorExecution, WorkflowExecution} -import edu.uci.ics.amber.engine.architecture.rpc.{AssignPortRequest, EmptyRequest, InitializeExecutorRequest, LinkWorkersRequest} +import edu.uci.ics.amber.engine.architecture.controller.{ + ControllerConfig, + ExecutionStatsUpdate, + WorkerAssignmentUpdate +} +import edu.uci.ics.amber.engine.architecture.controller.execution.{ + OperatorExecution, + WorkflowExecution +} +import edu.uci.ics.amber.engine.architecture.rpc.{ + AssignPortRequest, + EmptyRequest, + InitializeExecutorRequest, + LinkWorkersRequest +} import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.scheduling.config.{OperatorConfig, ResourceConfig} import edu.uci.ics.amber.engine.common.{AmberRuntime, PhysicalLink, WorkflowAggregatedState} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/ChannelMarkerManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/ChannelMarkerManager.scala index 7e7005c42a4..5a0ab559135 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/ChannelMarkerManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/ChannelMarkerManager.scala @@ -2,10 +2,7 @@ package edu.uci.ics.amber.engine.architecture.worker import edu.uci.ics.amber.engine.architecture.messaginglayer.InputGateway import edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerPayload -import edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.{ - NO_ALIGNMENT, - REQUIRE_ALIGNMENT -} +import edu.uci.ics.amber.engine.architecture.rpc.ChannelMarkerType.{NO_ALIGNMENT, REQUIRE_ALIGNMENT} import edu.uci.ics.amber.engine.common.{AmberLogging, CheckpointState} import edu.uci.ics.amber.engine.common.{ ActorVirtualIdentity, diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessor.scala index 78877be7f92..1e0877d1f66 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessor.scala @@ -18,11 +18,7 @@ import edu.uci.ics.amber.engine.architecture.rpc.{ } import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.MainThreadDelegateMessage import edu.uci.ics.amber.engine.architecture.worker.managers.SerializationManager -import edu.uci.ics.amber.engine.architecture.worker.WorkerState.{ - COMPLETED, - READY, - RUNNING -} +import edu.uci.ics.amber.engine.architecture.worker.WorkerState.{COMPLETED, READY, RUNNING} import edu.uci.ics.amber.engine.architecture.worker.WorkerStatistics import edu.uci.ics.amber.engine.common.ambermessage._ import edu.uci.ics.amber.engine.common.executor.OperatorExecutor diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/StatisticsManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/StatisticsManager.scala index 729e5da35b6..6a611e12922 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/StatisticsManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/StatisticsManager.scala @@ -1,9 +1,6 @@ package edu.uci.ics.amber.engine.architecture.worker.managers -import edu.uci.ics.amber.engine.architecture.worker.{ - PortTupleCountMapping, - WorkerStatistics -} +import edu.uci.ics.amber.engine.architecture.worker.{PortTupleCountMapping, WorkerStatistics} import edu.uci.ics.amber.engine.common.executor.{OperatorExecutor, SinkOperatorExecutor} import edu.uci.ics.amber.engine.common.PortIdentity diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddInputChannelHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddInputChannelHandler.scala index 585590208bd..f9c62c99497 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddInputChannelHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddInputChannelHandler.scala @@ -1,10 +1,7 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.{ - AddInputChannelRequest, - AsyncRPCContext -} +import edu.uci.ics.amber.engine.architecture.rpc.{AddInputChannelRequest, AsyncRPCContext} import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer import edu.uci.ics.amber.engine.architecture.worker.WorkerState.{PAUSED, READY, RUNNING} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddPartitioningHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddPartitioningHandler.scala index 31efb38f380..725944ec9a5 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddPartitioningHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AddPartitioningHandler.scala @@ -1,10 +1,7 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.{ - AddPartitioningRequest, - AsyncRPCContext -} +import edu.uci.ics.amber.engine.architecture.rpc.{AddPartitioningRequest, AsyncRPCContext} import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer import edu.uci.ics.amber.engine.architecture.worker.WorkerState.{PAUSED, READY, RUNNING} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AssignPortHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AssignPortHandler.scala index 08a721ba66a..78d7ebfeea4 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AssignPortHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/AssignPortHandler.scala @@ -1,10 +1,7 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.{ - AssignPortRequest, - AsyncRPCContext -} +import edu.uci.ics.amber.engine.architecture.rpc.{AssignPortRequest, AsyncRPCContext} import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer import edu.uci.ics.amber.engine.common.model.tuple.Schema diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/FinalizeCheckpointHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/FinalizeCheckpointHandler.scala index 5a86a66614e..c58b82d63f8 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/FinalizeCheckpointHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/FinalizeCheckpointHandler.scala @@ -1,10 +1,7 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.{ - AsyncRPCContext, - FinalizeCheckpointRequest -} +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, FinalizeCheckpointRequest} import edu.uci.ics.amber.engine.architecture.rpc.FinalizeCheckpointResponse import edu.uci.ics.amber.engine.architecture.worker.{ DataProcessorRPCHandlerInitializer, diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/InitializeExecutorHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/InitializeExecutorHandler.scala index ae87dcad52a..360cd097050 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/InitializeExecutorHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/InitializeExecutorHandler.scala @@ -3,10 +3,7 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo.generateJavaOpExec -import edu.uci.ics.amber.engine.architecture.rpc.{ - AsyncRPCContext, - InitializeExecutorRequest -} +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, InitializeExecutorRequest} import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer import edu.uci.ics.amber.engine.common.{AmberRuntime, VirtualIdentityUtils} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/PrepareCheckpointHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/PrepareCheckpointHandler.scala index e05d650d780..e9ca529426d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/PrepareCheckpointHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/PrepareCheckpointHandler.scala @@ -1,10 +1,7 @@ package edu.uci.ics.amber.engine.architecture.worker.promisehandlers import com.twitter.util.Future -import edu.uci.ics.amber.engine.architecture.rpc.{ - AsyncRPCContext, - PrepareCheckpointRequest -} +import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, PrepareCheckpointRequest} import edu.uci.ics.amber.engine.architecture.rpc.EmptyReturn import edu.uci.ics.amber.engine.architecture.worker.{ DataProcessorRPCHandlerInitializer, diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/client/ClientActor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/client/ClientActor.scala index 6ff276869a6..a14ff214bb3 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/client/ClientActor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/client/ClientActor.scala @@ -11,11 +11,7 @@ import edu.uci.ics.amber.engine.architecture.common.WorkflowActor.{ } import edu.uci.ics.amber.engine.architecture.controller.{ClientEvent, Controller, ControllerConfig} import edu.uci.ics.amber.engine.architecture.rpc.{AsyncRPCContext, ControlRequest} -import edu.uci.ics.amber.engine.architecture.rpc.{ - ControlException, - ControlReturn, - ReturnInvocation -} +import edu.uci.ics.amber.engine.architecture.rpc.{ControlException, ControlReturn, ReturnInvocation} import edu.uci.ics.amber.engine.common.ambermessage.WorkflowMessage.getInMemSize import edu.uci.ics.amber.engine.common.AmberLogging import edu.uci.ics.amber.engine.common.ambermessage.{ diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/PhysicalPlan.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/PhysicalPlan.scala index b79c8358691..215b09ff4ac 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/PhysicalPlan.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/common/model/PhysicalPlan.scala @@ -3,11 +3,7 @@ package edu.uci.ics.amber.engine.common.model import com.fasterxml.jackson.annotation.JsonIgnore import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.common.VirtualIdentityUtils -import edu.uci.ics.amber.engine.common.{ - ActorVirtualIdentity, - OperatorIdentity, - PhysicalOpIdentity -} +import edu.uci.ics.amber.engine.common.{ActorVirtualIdentity, OperatorIdentity, PhysicalOpIdentity} import edu.uci.ics.amber.engine.common.PhysicalLink import edu.uci.ics.texera.workflow.common.workflow.{LogicalPlan, PartitionInfo, UnknownPartition} import org.jgrapht.alg.connectivity.BiconnectivityInspector diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/TexeraWebApplication.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/TexeraWebApplication.scala index 5c1e97919db..b11c34174d4 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/TexeraWebApplication.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/TexeraWebApplication.scala @@ -15,7 +15,12 @@ import edu.uci.ics.amber.engine.common.ExecutionIdentity import Utils.{maptoStatusCode, objectMapper} import edu.uci.ics.amber.engine.common.WorkflowAggregatedState.{COMPLETED, FAILED} import edu.uci.ics.texera.web.auth.JwtAuth.jwtConsumer -import edu.uci.ics.texera.web.auth.{GuestAuthFilter, SessionUser, UserAuthenticator, UserRoleAuthorizer} +import edu.uci.ics.texera.web.auth.{ + GuestAuthFilter, + SessionUser, + UserAuthenticator, + UserRoleAuthorizer +} import edu.uci.ics.texera.web.model.jooq.generated.tables.pojos.WorkflowExecutions import edu.uci.ics.texera.web.resource.auth.{AuthResource, GoogleAuthResource} import edu.uci.ics.texera.web.resource._ @@ -23,14 +28,29 @@ import edu.uci.ics.texera.web.resource.dashboard.DashboardResource import edu.uci.ics.texera.web.resource.dashboard.admin.execution.AdminExecutionResource import edu.uci.ics.texera.web.resource.dashboard.admin.user.AdminUserResource import edu.uci.ics.texera.web.resource.dashboard.hub.workflow.HubWorkflowResource -import edu.uci.ics.texera.web.resource.dashboard.user.dataset.{DatasetAccessResource, DatasetResource} -import edu.uci.ics.texera.web.resource.dashboard.user.dataset.`type`.{DatasetFileNode, DatasetFileNodeSerializer} +import edu.uci.ics.texera.web.resource.dashboard.user.dataset.{ + DatasetAccessResource, + DatasetResource +} +import edu.uci.ics.texera.web.resource.dashboard.user.dataset.`type`.{ + DatasetFileNode, + DatasetFileNodeSerializer +} import edu.uci.ics.texera.web.resource.dashboard.user.dataset.service.GitVersionControlLocalFileStorage import edu.uci.ics.texera.web.resource.dashboard.user.dataset.utils.PathUtils.getAllDatasetDirectories -import edu.uci.ics.texera.web.resource.dashboard.user.project.{ProjectAccessResource, ProjectResource, PublicProjectResource} +import edu.uci.ics.texera.web.resource.dashboard.user.project.{ + ProjectAccessResource, + ProjectResource, + PublicProjectResource +} import edu.uci.ics.texera.web.resource.dashboard.user.quota.UserQuotaResource import edu.uci.ics.texera.web.resource.dashboard.user.discussion.UserDiscussionResource -import edu.uci.ics.texera.web.resource.dashboard.user.workflow.{WorkflowAccessResource, WorkflowExecutionsResource, WorkflowResource, WorkflowVersionResource} +import edu.uci.ics.texera.web.resource.dashboard.user.workflow.{ + WorkflowAccessResource, + WorkflowExecutionsResource, + WorkflowResource, + WorkflowVersionResource +} import edu.uci.ics.texera.web.resource.languageserver.PythonLanguageServerManager import edu.uci.ics.texera.web.service.ExecutionsMetadataPersistService import edu.uci.ics.texera.web.storage.MongoDatabaseManager diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/WorkflowLifecycleManager.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/WorkflowLifecycleManager.scala index 66ff9ac316a..d19f6e64769 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/WorkflowLifecycleManager.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/WorkflowLifecycleManager.scala @@ -4,7 +4,11 @@ import java.time.{LocalDateTime, Duration => JDuration} import akka.actor.Cancellable import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.common.WorkflowAggregatedState.RUNNING -import edu.uci.ics.amber.engine.common.{AmberRuntime, ExecutionMetadataStore, WorkflowAggregatedState} +import edu.uci.ics.amber.engine.common.{ + AmberRuntime, + ExecutionMetadataStore, + WorkflowAggregatedState +} import edu.uci.ics.texera.web.storage.ExecutionStateStore import scala.concurrent.duration.DurationInt diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowWebsocketResource.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowWebsocketResource.scala index 9f62763d22a..bd9cb8d006d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowWebsocketResource.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowWebsocketResource.scala @@ -8,7 +8,11 @@ import edu.uci.ics.amber.engine.common.WorkflowIdentity import edu.uci.ics.amber.error.ErrorUtils.getStackTraceWithAllCauses import edu.uci.ics.amber.engine.common.Utils.objectMapper import edu.uci.ics.texera.web.model.jooq.generated.tables.pojos.User -import edu.uci.ics.texera.web.model.websocket.event.{CacheStatusUpdateEvent, WorkflowErrorEvent, WorkflowStateEvent} +import edu.uci.ics.texera.web.model.websocket.event.{ + CacheStatusUpdateEvent, + WorkflowErrorEvent, + WorkflowStateEvent +} import edu.uci.ics.texera.web.model.websocket.request._ import edu.uci.ics.texera.web.model.websocket.response._ import edu.uci.ics.texera.web.service.{WorkflowCacheChecker, WorkflowService} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/EmailNotificationService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/EmailNotificationService.scala index 954284179bf..e13952b2537 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/EmailNotificationService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/EmailNotificationService.scala @@ -1,6 +1,5 @@ package edu.uci.ics.texera.web.service - import edu.uci.ics.amber.engine.common.WorkflowAggregatedState import java.util.concurrent.{ExecutorService, Executors} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionConsoleService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionConsoleService.scala index e7c9747209b..a6416f91b85 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionConsoleService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionConsoleService.scala @@ -19,11 +19,7 @@ import edu.uci.ics.texera.web.model.websocket.request.python.{ } import edu.uci.ics.texera.web.model.websocket.response.python.PythonExpressionEvaluateResponse import edu.uci.ics.texera.web.storage.ExecutionStateStore -import edu.uci.ics.amber.engine.common.{ - EvaluatedValueList, - ExecutionConsoleStore, - OperatorConsole -} +import edu.uci.ics.amber.engine.common.{EvaluatedValueList, ExecutionConsoleStore, OperatorConsole} import edu.uci.ics.texera.web.model.websocket.request.RetryRequest import edu.uci.ics.texera.web.{SubscriptionManager, WebsocketInput} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionResultService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionResultService.scala index af1ae2bafd7..618022c4258 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionResultService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionResultService.scala @@ -11,10 +11,19 @@ import edu.uci.ics.amber.engine.common.client.AmberClient import edu.uci.ics.amber.engine.common.model.tuple.Tuple import edu.uci.ics.amber.engine.common.OperatorIdentity import edu.uci.ics.amber.engine.common.IncrementalOutputMode.{SET_DELTA, SET_SNAPSHOT} -import edu.uci.ics.texera.web.model.websocket.event.{PaginatedResultEvent, TexeraWebSocketEvent, WebResultUpdateEvent} +import edu.uci.ics.texera.web.model.websocket.event.{ + PaginatedResultEvent, + TexeraWebSocketEvent, + WebResultUpdateEvent +} import edu.uci.ics.texera.web.model.websocket.request.ResultPaginationRequest import edu.uci.ics.texera.web.service.ExecutionResultService.WebResultUpdate -import edu.uci.ics.texera.web.storage.{ExecutionStateStore, OperatorResultMetadata, WorkflowResultStore, WorkflowStateStore} +import edu.uci.ics.texera.web.storage.{ + ExecutionStateStore, + OperatorResultMetadata, + WorkflowResultStore, + WorkflowStateStore +} import edu.uci.ics.amber.engine.common.ExecutionMetadataStore import edu.uci.ics.amber.engine.common.WorkflowAggregatedState._ import edu.uci.ics.texera.web.SubscriptionManager diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionRuntimeService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionRuntimeService.scala index 8018e9adddc..d0daff24baf 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionRuntimeService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionRuntimeService.scala @@ -8,7 +8,13 @@ import edu.uci.ics.amber.engine.common.client.AmberClient import edu.uci.ics.amber.engine.common.ChannelMarkerIdentity import edu.uci.ics.amber.engine.common.WorkflowAggregatedState._ import edu.uci.ics.texera.web.{SubscriptionManager, WebsocketInput} -import edu.uci.ics.texera.web.model.websocket.request.{SkipTupleRequest, WorkflowCheckpointRequest, WorkflowKillRequest, WorkflowPauseRequest, WorkflowResumeRequest} +import edu.uci.ics.texera.web.model.websocket.request.{ + SkipTupleRequest, + WorkflowCheckpointRequest, + WorkflowKillRequest, + WorkflowPauseRequest, + WorkflowResumeRequest +} import edu.uci.ics.texera.web.storage.ExecutionStateStore import edu.uci.ics.texera.web.storage.ExecutionStateStore.updateWorkflowState diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala index ca14763469f..4976a68bc55 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala @@ -2,9 +2,22 @@ package edu.uci.ics.texera.web.service import com.google.protobuf.timestamp.Timestamp import com.typesafe.scalalogging.LazyLogging -import edu.uci.ics.amber.engine.architecture.controller.{ExecutionStatsUpdate, FatalError, WorkerAssignmentUpdate, WorkflowRecoveryStatus} +import edu.uci.ics.amber.engine.architecture.controller.{ + ExecutionStatsUpdate, + FatalError, + WorkerAssignmentUpdate, + WorkflowRecoveryStatus +} import edu.uci.ics.amber.engine.architecture.worker.PortTupleCountMapping -import edu.uci.ics.amber.engine.common.{AmberConfig, OperatorMetrics, OperatorStatistics, OperatorWorkerMapping, Utils, WorkflowAggregatedState, WorkflowFatalError} +import edu.uci.ics.amber.engine.common.{ + AmberConfig, + OperatorMetrics, + OperatorStatistics, + OperatorWorkerMapping, + Utils, + WorkflowAggregatedState, + WorkflowFatalError +} import edu.uci.ics.amber.engine.common.client.AmberClient import edu.uci.ics.amber.engine.common.model.WorkflowContext import edu.uci.ics.amber.error.ErrorUtils.{getOperatorFromActorIdOpt, getStackTraceWithAllCauses} @@ -12,7 +25,12 @@ import Utils.maptoStatusCode import edu.uci.ics.texera.web.model.jooq.generated.tables.pojos.WorkflowRuntimeStatistics import edu.uci.ics.texera.web.model.jooq.generated.tables.daos.WorkflowRuntimeStatisticsDao import edu.uci.ics.texera.web.{SqlServer, SubscriptionManager} -import edu.uci.ics.texera.web.model.websocket.event.{ExecutionDurationUpdateEvent, OperatorAggregatedMetrics, OperatorStatisticsUpdateEvent, WorkerAssignmentUpdateEvent} +import edu.uci.ics.texera.web.model.websocket.event.{ + ExecutionDurationUpdateEvent, + OperatorAggregatedMetrics, + OperatorStatisticsUpdateEvent, + WorkerAssignmentUpdateEvent +} import edu.uci.ics.texera.web.storage.ExecutionStateStore import edu.uci.ics.texera.web.storage.ExecutionStateStore.updateWorkflowState import edu.uci.ics.amber.engine.common.FatalErrorType.EXECUTION_FAILURE diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/FriesReconfigurationAlgorithm.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/FriesReconfigurationAlgorithm.scala index b60a4046d6c..591ae0d8d05 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/FriesReconfigurationAlgorithm.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/FriesReconfigurationAlgorithm.scala @@ -1,9 +1,6 @@ package edu.uci.ics.texera.web.service -import edu.uci.ics.amber.engine.architecture.rpc.{ - ModifyLogicRequest, - PropagateChannelMarkerRequest -} +import edu.uci.ics.amber.engine.architecture.rpc.{ModifyLogicRequest, PropagateChannelMarkerRequest} import edu.uci.ics.amber.engine.architecture.scheduling.{Region, WorkflowExecutionCoordinator} import edu.uci.ics.amber.engine.common.model.PhysicalPlan import edu.uci.ics.amber.engine.common.PhysicalOpIdentity diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowExecutionService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowExecutionService.scala index e898ca68e3e..a17ef028d73 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowExecutionService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowExecutionService.scala @@ -6,7 +6,11 @@ import edu.uci.ics.amber.engine.architecture.rpc.EmptyRequest import edu.uci.ics.amber.engine.common.Utils import edu.uci.ics.amber.engine.common.client.AmberClient import edu.uci.ics.amber.engine.common.model.WorkflowContext -import edu.uci.ics.texera.web.model.websocket.event.{TexeraWebSocketEvent, WorkflowErrorEvent, WorkflowStateEvent} +import edu.uci.ics.texera.web.model.websocket.event.{ + TexeraWebSocketEvent, + WorkflowErrorEvent, + WorkflowStateEvent +} import edu.uci.ics.texera.web.model.websocket.request.WorkflowExecuteRequest import edu.uci.ics.texera.web.storage.ExecutionStateStore import edu.uci.ics.texera.web.storage.ExecutionStateStore.updateWorkflowState diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowService.scala index b971016e0dc..3d5e027355e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowService.scala @@ -3,7 +3,10 @@ package edu.uci.ics.texera.web.service import com.google.protobuf.timestamp.Timestamp import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.engine.architecture.controller.ControllerConfig -import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.{FaultToleranceConfig, StateRestoreConfig} +import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.{ + FaultToleranceConfig, + StateRestoreConfig +} import edu.uci.ics.amber.engine.common.AmberConfig import edu.uci.ics.amber.engine.common.amberexception.WorkflowRuntimeException import edu.uci.ics.amber.engine.common.model.WorkflowContext diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/ExecutionStateStore.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/ExecutionStateStore.scala index 484db4865ab..ac7940261af 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/ExecutionStateStore.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/storage/ExecutionStateStore.scala @@ -2,7 +2,13 @@ package edu.uci.ics.texera.web.storage import edu.uci.ics.amber.engine.common.Utils.maptoStatusCode import edu.uci.ics.texera.web.service.ExecutionsMetadataPersistService -import edu.uci.ics.amber.engine.common.{ExecutionBreakpointStore, ExecutionConsoleStore, ExecutionMetadataStore, ExecutionStatsStore, WorkflowAggregatedState} +import edu.uci.ics.amber.engine.common.{ + ExecutionBreakpointStore, + ExecutionConsoleStore, + ExecutionMetadataStore, + ExecutionStatsStore, + WorkflowAggregatedState +} import java.sql.Timestamp diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/LogicalOp.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/LogicalOp.scala index 675f985e121..97e56cf1f29 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/LogicalOp.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/common/operators/LogicalOp.scala @@ -12,11 +12,7 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.common.executor.OperatorExecutor import edu.uci.ics.amber.engine.common.model.{PhysicalOp, PhysicalPlan, WorkflowContext} import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.{ - ExecutionIdentity, - OperatorIdentity, - WorkflowIdentity -} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, OperatorIdentity, WorkflowIdentity} import edu.uci.ics.amber.engine.common.PortIdentity import edu.uci.ics.texera.web.OPversion import edu.uci.ics.texera.workflow.common.metadata.{OperatorInfo, PropertyNameConstants} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/aggregate/AggregateOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/aggregate/AggregateOpDesc.scala index ea1bf8a5a7b..27fc067277f 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/aggregate/AggregateOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/aggregate/AggregateOpDesc.scala @@ -5,11 +5,7 @@ import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, PhysicalPlan, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.{ - ExecutionIdentity, - PhysicalOpIdentity, - WorkflowIdentity -} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, PhysicalOpIdentity, WorkflowIdentity} import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PhysicalLink, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.AutofillAttributeNameList import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/hashJoin/HashJoinOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/hashJoin/HashJoinOpDesc.scala index 635735c1fb5..16b8224a48d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/hashJoin/HashJoinOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/hashJoin/HashJoinOpDesc.scala @@ -6,11 +6,7 @@ import com.kjetland.jackson.jsonSchema.annotations.{JsonSchemaInject, JsonSchema import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, PhysicalPlan, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.{Attribute, AttributeType, Schema} -import edu.uci.ics.amber.engine.common.{ - ExecutionIdentity, - PhysicalOpIdentity, - WorkflowIdentity -} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, PhysicalOpIdentity, WorkflowIdentity} import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PhysicalLink, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.annotations.{ AutofillAttributeName, diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/cache/CacheSourceOpDesc.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/cache/CacheSourceOpDesc.scala index 738eebef87d..a80277e1126 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/cache/CacheSourceOpDesc.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/operators/source/cache/CacheSourceOpDesc.scala @@ -3,11 +3,7 @@ package edu.uci.ics.texera.workflow.operators.source.cache import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.OpExecInitInfo import edu.uci.ics.amber.engine.common.model.{PhysicalOp, SchemaPropagationFunc} import edu.uci.ics.amber.engine.common.model.tuple.Schema -import edu.uci.ics.amber.engine.common.{ - ExecutionIdentity, - OperatorIdentity, - WorkflowIdentity -} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, OperatorIdentity, WorkflowIdentity} import edu.uci.ics.amber.engine.common.OutputPort import edu.uci.ics.texera.workflow.common.metadata.{OperatorGroupConstants, OperatorInfo} import edu.uci.ics.texera.workflow.common.operators.source.SourceOperatorDescriptor diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/TrivialControlSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/TrivialControlSpec.scala index 64759fda08f..f5f85fbd938 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/TrivialControlSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/control/TrivialControlSpec.scala @@ -10,11 +10,7 @@ import edu.uci.ics.amber.engine.architecture.common.WorkflowActor.{ } import edu.uci.ics.amber.engine.architecture.control.utils.TrivialControlTester import edu.uci.ics.amber.engine.architecture.rpc._ -import edu.uci.ics.amber.engine.architecture.rpc.{ - IntResponse, - ReturnInvocation, - StringResponse -} +import edu.uci.ics.amber.engine.architecture.rpc.{IntResponse, ReturnInvocation, StringResponse} import edu.uci.ics.amber.engine.architecture.rpc.RPCTesterGrpc.{ METHOD_SEND_CHAIN, METHOD_SEND_COLLECT, diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/CheckpointSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/CheckpointSpec.scala index 83d42b3b0a2..ca635b20119 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/CheckpointSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/faulttolerance/CheckpointSpec.scala @@ -4,8 +4,15 @@ import akka.actor.{ActorSystem, Props} import akka.serialization.SerializationExtension import com.twitter.util.{Await, Duration} import edu.uci.ics.amber.clustering.SingleNodeListener -import edu.uci.ics.amber.engine.architecture.controller.{ControllerConfig, ControllerProcessor, ExecutionStateUpdate} -import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.{OpExecInitInfoWithCode, OpExecInitInfoWithFunc} +import edu.uci.ics.amber.engine.architecture.controller.{ + ControllerConfig, + ControllerProcessor, + ExecutionStateUpdate +} +import edu.uci.ics.amber.engine.architecture.deploysemantics.layer.{ + OpExecInitInfoWithCode, + OpExecInitInfoWithFunc +} import edu.uci.ics.amber.engine.architecture.rpc.{EmptyRequest, TakeGlobalCheckpointRequest} import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.StateRestoreConfig import edu.uci.ics.amber.engine.architecture.worker.DataProcessor diff --git a/core/amber/src/test/scala/edu/uci/ics/texera/workflow/common/workflow/SchemaPropagationSpec.scala b/core/amber/src/test/scala/edu/uci/ics/texera/workflow/common/workflow/SchemaPropagationSpec.scala index 74ef5053045..c13b6ea4865 100644 --- a/core/amber/src/test/scala/edu/uci/ics/texera/workflow/common/workflow/SchemaPropagationSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/texera/workflow/common/workflow/SchemaPropagationSpec.scala @@ -2,11 +2,7 @@ package edu.uci.ics.texera.workflow.common.workflow import edu.uci.ics.amber.engine.common.model.{PhysicalOp, WorkflowContext} import edu.uci.ics.amber.engine.common.model.tuple.{AttributeType, Schema} -import edu.uci.ics.amber.engine.common.{ - ExecutionIdentity, - OperatorIdentity, - WorkflowIdentity -} +import edu.uci.ics.amber.engine.common.{ExecutionIdentity, OperatorIdentity, WorkflowIdentity} import edu.uci.ics.amber.engine.common.{InputPort, OutputPort, PortIdentity} import edu.uci.ics.texera.workflow.common.metadata.OperatorInfo import edu.uci.ics.texera.workflow.common.operators.LogicalOp