├── .asf.yaml ├── .editorconfig ├── .github ├── CODE_OF_CONDUCT.md ├── COMMITTING.md ├── CONTRIBUTING.md ├── pull_request_template.md └── workflows │ ├── README.md │ ├── maven.yml │ ├── node.js.yml │ └── sonarcloud.yml ├── .gitignore ├── DISCLAIMER ├── LICENSE ├── NOTICE ├── README.md ├── bin ├── generate_javadocs.sh ├── install_nemo.sh ├── json2dot.py ├── metric-parser.py ├── run_beam.sh ├── run_nexmark.sh ├── run_spark.sh ├── run_webserver.sh ├── sonar_qube.sh └── xgboost_optimization.sh ├── checkstyle.license ├── checkstyle.xml ├── client ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ ├── nemo │ │ └── client │ │ │ ├── ClientEndpoint.java │ │ │ ├── ClientUtils.java │ │ │ ├── DriverEndpoint.java │ │ │ ├── DriverRPCServer.java │ │ │ ├── JobLauncher.java │ │ │ ├── NemoClient.java │ │ │ ├── StateTranslator.java │ │ │ └── beam │ │ │ ├── BeamStateTranslator.java │ │ │ ├── NemoPipelineResult.java │ │ │ ├── NemoRunner.java │ │ │ └── NemoRunnerRegistrar.java │ │ └── reef │ │ └── runtime │ │ └── yarn │ │ └── ClassPathBuilder.java │ └── test │ └── java │ └── org │ └── apache │ └── nemo │ └── client │ ├── ClientDriverRPCTest.java │ └── ClientEndpointTest.java ├── common ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── nemo │ │ │ └── common │ │ │ ├── ByteBufferInputStream.java │ │ │ ├── Cloneable.java │ │ │ ├── HashRange.java │ │ │ ├── KeyExtractor.java │ │ │ ├── KeyRange.java │ │ │ ├── Pair.java │ │ │ ├── PairKeyExtractor.java │ │ │ ├── StateMachine.java │ │ │ ├── Util.java │ │ │ ├── coder │ │ │ ├── BytesDecoderFactory.java │ │ │ ├── BytesEncoderFactory.java │ │ │ ├── DecoderFactory.java │ │ │ ├── EncoderFactory.java │ │ │ ├── IntDecoderFactory.java │ │ │ ├── IntEncoderFactory.java │ │ │ ├── LongDecoderFactory.java │ │ │ ├── LongEncoderFactory.java │ │ │ ├── PairDecoderFactory.java │ │ │ └── PairEncoderFactory.java │ │ │ ├── dag │ │ │ ├── DAG.java │ │ │ ├── DAGBuilder.java │ │ │ ├── DAGInterface.java │ │ │ ├── Edge.java │ │ │ └── Vertex.java │ │ │ ├── eventhandler │ │ │ ├── CommonEventHandler.java │ │ │ ├── CompilerEvent.java │ │ │ ├── CompilerEventHandler.java │ │ │ ├── PubSubEventHandlerWrapper.java │ │ │ ├── RuntimeEvent.java │ │ │ └── RuntimeEventHandler.java │ │ │ ├── exception │ │ │ ├── BlockFetchException.java │ │ │ ├── BlockWriteException.java │ │ │ ├── CompileTimeOptimizationException.java │ │ │ ├── ContainerException.java │ │ │ ├── DataSourceException.java │ │ │ ├── DynamicOptimizationException.java │ │ │ ├── IllegalEdgeOperationException.java │ │ │ ├── IllegalMessageException.java │ │ │ ├── IllegalStateTransitionException.java │ │ │ ├── IllegalVertexOperationException.java │ │ │ ├── InvalidParameterException.java │ │ │ ├── InvalidUserMainException.java │ │ │ ├── JsonParseException.java │ │ │ ├── MetricException.java │ │ │ ├── NodeConnectionException.java │ │ │ ├── OutputMismatchException.java │ │ │ ├── PhysicalPlanGenerationException.java │ │ │ ├── RuntimeOptimizationException.java │ │ │ ├── SchedulingException.java │ │ │ ├── SimulationException.java │ │ │ ├── UnknownExecutionStateException.java │ │ │ ├── UnknownFailureCauseException.java │ │ │ ├── UnrecoverableFailureException.java │ │ │ ├── UnsupportedBlockStoreException.java │ │ │ ├── UnsupportedCommPatternException.java │ │ │ ├── UnsupportedCompressionException.java │ │ │ ├── UnsupportedExecutionPropertyException.java │ │ │ ├── UnsupportedMethodException.java │ │ │ ├── UnsupportedMetricException.java │ │ │ └── UnsupportedPartitionerException.java │ │ │ ├── ir │ │ │ ├── BoundedIteratorReadable.java │ │ │ ├── IRDAG.java │ │ │ ├── IRDAGChecker.java │ │ │ ├── IdManager.java │ │ │ ├── OutputCollector.java │ │ │ ├── Readable.java │ │ │ ├── edge │ │ │ │ ├── IREdge.java │ │ │ │ └── executionproperty │ │ │ │ │ ├── AdditionalOutputTagProperty.java │ │ │ │ │ ├── BlockFetchFailureProperty.java │ │ │ │ │ ├── CacheIDProperty.java │ │ │ │ │ ├── CommunicationPatternProperty.java │ │ │ │ │ ├── CompressionProperty.java │ │ │ │ │ ├── DataFlowProperty.java │ │ │ │ │ ├── DataPersistenceProperty.java │ │ │ │ │ ├── DataStoreProperty.java │ │ │ │ │ ├── DecoderProperty.java │ │ │ │ │ ├── DecompressionProperty.java │ │ │ │ │ ├── DuplicateEdgeGroupProperty.java │ │ │ │ │ ├── DuplicateEdgeGroupPropertyValue.java │ │ │ │ │ ├── EncoderProperty.java │ │ │ │ │ ├── KeyDecoderProperty.java │ │ │ │ │ ├── KeyEncoderProperty.java │ │ │ │ │ ├── KeyExtractorProperty.java │ │ │ │ │ ├── MessageIdEdgeProperty.java │ │ │ │ │ ├── PartitionSetProperty.java │ │ │ │ │ ├── PartitionerProperty.java │ │ │ │ │ └── SubPartitionSetProperty.java │ │ │ ├── executionproperty │ │ │ │ ├── AssociatedProperty.java │ │ │ │ ├── EdgeExecutionProperty.java │ │ │ │ ├── ExecutionProperty.java │ │ │ │ ├── ExecutionPropertyMap.java │ │ │ │ ├── ResourceSpecification.java │ │ │ │ └── VertexExecutionProperty.java │ │ │ └── vertex │ │ │ │ ├── CachedSourceVertex.java │ │ │ │ ├── IRVertex.java │ │ │ │ ├── InMemorySourceVertex.java │ │ │ │ ├── LoopVertex.java │ │ │ │ ├── OperatorVertex.java │ │ │ │ ├── SourceVertex.java │ │ │ │ ├── executionproperty │ │ │ │ ├── ClonedSchedulingProperty.java │ │ │ │ ├── EnableDynamicTaskSizingProperty.java │ │ │ │ ├── IgnoreSchedulingTempDataReceiverProperty.java │ │ │ │ ├── MessageIdVertexProperty.java │ │ │ │ ├── ParallelismProperty.java │ │ │ │ ├── ResourceAntiAffinityProperty.java │ │ │ │ ├── ResourceLambdaProperty.java │ │ │ │ ├── ResourceLocalityProperty.java │ │ │ │ ├── ResourcePriorityProperty.java │ │ │ │ ├── ResourceSiteProperty.java │ │ │ │ ├── ResourceSlotProperty.java │ │ │ │ └── ScheduleGroupProperty.java │ │ │ │ ├── transform │ │ │ │ ├── LatencymarkEmitTransform.java │ │ │ │ ├── MessageAggregatorTransform.java │ │ │ │ ├── MessageGeneratorTransform.java │ │ │ │ ├── NoWatermarkEmitTransform.java │ │ │ │ ├── SignalTransform.java │ │ │ │ ├── StreamTransform.java │ │ │ │ └── Transform.java │ │ │ │ └── utility │ │ │ │ ├── RelayVertex.java │ │ │ │ ├── SamplingVertex.java │ │ │ │ ├── TaskSizeSplitterVertex.java │ │ │ │ └── runtimepass │ │ │ │ ├── MessageAggregatorVertex.java │ │ │ │ ├── MessageGeneratorVertex.java │ │ │ │ └── SignalVertex.java │ │ │ ├── partitioner │ │ │ ├── DedicatedKeyPerElement.java │ │ │ ├── DedicatedKeyPerElementPartitioner.java │ │ │ ├── HashPartitioner.java │ │ │ ├── IntactPartitioner.java │ │ │ └── Partitioner.java │ │ │ ├── pass │ │ │ └── Pass.java │ │ │ ├── punctuation │ │ │ ├── Finishmark.java │ │ │ ├── LatencyMark.java │ │ │ └── Watermark.java │ │ │ └── test │ │ │ ├── ArgBuilder.java │ │ │ ├── EmptyComponents.java │ │ │ ├── ExampleTestArgs.java │ │ │ └── ExampleTestUtil.java │ └── resources │ │ └── log4j.properties │ └── test │ └── java │ └── org │ └── apache │ └── nemo │ └── common │ ├── DAGTest.java │ ├── PairTest.java │ ├── StateMachineTest.java │ ├── UtilTest.java │ ├── coder │ └── CoderFactoryTest.java │ └── ir │ ├── IRDAGTest.java │ ├── LoopVertexTest.java │ └── executionproperty │ └── ExecutionPropertyMapTest.java ├── compiler ├── backend │ ├── pom.xml │ └── src │ │ ├── main │ │ └── java │ │ │ └── org │ │ │ └── apache │ │ │ └── nemo │ │ │ └── compiler │ │ │ └── backend │ │ │ ├── Backend.java │ │ │ └── nemo │ │ │ ├── NemoBackend.java │ │ │ ├── NemoPlanRewriter.java │ │ │ └── prophet │ │ │ ├── ParallelismProphet.java │ │ │ ├── Prophet.java │ │ │ └── SkewProphet.java │ │ └── test │ │ └── java │ │ └── org │ │ └── apache │ │ └── nemo │ │ └── compiler │ │ └── backend │ │ └── nemo │ │ └── NemoBackendTest.java ├── frontend │ ├── beam │ │ ├── pom.xml │ │ └── src │ │ │ └── main │ │ │ └── java │ │ │ └── org │ │ │ └── apache │ │ │ └── nemo │ │ │ └── compiler │ │ │ └── frontend │ │ │ └── beam │ │ │ ├── BeamKeyExtractor.java │ │ │ ├── InMemorySideInputReader.java │ │ │ ├── NemoPipelineOptions.java │ │ │ ├── PipelineTranslationContext.java │ │ │ ├── PipelineTranslator.java │ │ │ ├── PipelineVisitor.java │ │ │ ├── SideInputElement.java │ │ │ ├── coder │ │ │ ├── BeamDecoderFactory.java │ │ │ ├── BeamEncoderFactory.java │ │ │ └── SideInputCoder.java │ │ │ ├── source │ │ │ ├── BeamBoundedSourceVertex.java │ │ │ └── BeamUnboundedSourceVertex.java │ │ │ └── transform │ │ │ ├── AbstractDoFnTransform.java │ │ │ ├── CombineFnFinalTransform.java │ │ │ ├── CombineFnPartialTransform.java │ │ │ ├── CreateViewTransform.java │ │ │ ├── DefaultOutputManager.java │ │ │ ├── DoFnTransform.java │ │ │ ├── FinalCombineFn.java │ │ │ ├── FlattenTransform.java │ │ │ ├── GBKTransform.java │ │ │ ├── GroupByKeyTransform.java │ │ │ ├── InMemoryStateInternalsFactory.java │ │ │ ├── InMemoryTimerInternalsFactory.java │ │ │ ├── LoopCompositeTransform.java │ │ │ ├── PartialCombineFn.java │ │ │ ├── PushBackDoFnTransform.java │ │ │ ├── SideInputTransform.java │ │ │ └── WindowFnTransform.java │ └── spark │ │ ├── pom.xml │ │ └── src │ │ └── main │ │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── nemo │ │ │ └── compiler │ │ │ └── frontend │ │ │ └── spark │ │ │ ├── SparkBroadcastVariables.java │ │ │ ├── SparkKeyExtractor.java │ │ │ ├── coder │ │ │ ├── SparkDecoderFactory.java │ │ │ └── SparkEncoderFactory.java │ │ │ ├── core │ │ │ ├── JavaSparkContext.java │ │ │ ├── SparkBroadcast.java │ │ │ ├── SparkContext.java │ │ │ ├── SparkFrontendUtils.java │ │ │ └── rdd │ │ │ │ ├── SparkJavaPairRDD.java │ │ │ │ └── SparkJavaRDD.java │ │ │ ├── source │ │ │ ├── SparkDatasetBoundedSourceVertex.java │ │ │ ├── SparkSourceUtil.java │ │ │ └── SparkTextFileBoundedSourceVertex.java │ │ │ ├── sql │ │ │ ├── DataFrameReader.java │ │ │ ├── Dataset.java │ │ │ ├── NemoSparkUserFacingClass.java │ │ │ └── SparkSession.java │ │ │ └── transform │ │ │ ├── CollectTransform.java │ │ │ ├── FlatMapTransform.java │ │ │ ├── GroupByKeyTransform.java │ │ │ ├── HDFSTextFileTransform.java │ │ │ ├── LocalTextFileTransform.java │ │ │ ├── MapToPairTransform.java │ │ │ ├── MapTransform.java │ │ │ ├── ReduceByKeyTransform.java │ │ │ └── ReduceTransform.java │ │ └── scala │ │ └── org │ │ └── apache │ │ └── nemo │ │ └── compiler │ │ └── frontend │ │ └── spark │ │ └── core │ │ └── rdd │ │ ├── PairRDDFunctions.scala │ │ └── RDD.scala ├── optimizer │ ├── pom.xml │ └── src │ │ ├── main │ │ └── java │ │ │ └── org │ │ │ └── apache │ │ │ └── nemo │ │ │ └── compiler │ │ │ └── optimizer │ │ │ ├── NemoOptimizer.java │ │ │ ├── Optimizer.java │ │ │ ├── OptimizerUtils.java │ │ │ ├── examples │ │ │ └── MapReduceDisaggregationOptimization.java │ │ │ ├── pass │ │ │ ├── compiletime │ │ │ │ ├── CompileTimePass.java │ │ │ │ ├── Requires.java │ │ │ │ ├── annotating │ │ │ │ │ ├── AggressiveSpeculativeCloningPass.java │ │ │ │ │ ├── Annotates.java │ │ │ │ │ ├── AnnotatingPass.java │ │ │ │ │ ├── CompressionPass.java │ │ │ │ │ ├── DefaultDataPersistencePass.java │ │ │ │ │ ├── DefaultDataStorePass.java │ │ │ │ │ ├── DefaultEdgeDecoderPass.java │ │ │ │ │ ├── DefaultEdgeEncoderPass.java │ │ │ │ │ ├── DefaultParallelismPass.java │ │ │ │ │ ├── DefaultScheduleGroupPass.java │ │ │ │ │ ├── DisaggregationEdgeDataStorePass.java │ │ │ │ │ ├── DuplicateEdgeGroupSizePass.java │ │ │ │ │ ├── LambdaPass.java │ │ │ │ │ ├── LargeShuffleAnnotatingPass.java │ │ │ │ │ ├── PipeTransferForAllEdgesPass.java │ │ │ │ │ ├── ResourceLocalityPass.java │ │ │ │ │ ├── ResourceSitePass.java │ │ │ │ │ ├── ResourceSlotPass.java │ │ │ │ │ ├── ShuffleEdgePushPass.java │ │ │ │ │ ├── SkewAnnotatingPass.java │ │ │ │ │ ├── TransientResourceDataTransferPass.java │ │ │ │ │ ├── TransientResourcePriorityPass.java │ │ │ │ │ ├── UpfrontCloningPass.java │ │ │ │ │ └── XGBoostPass.java │ │ │ │ ├── composite │ │ │ │ │ ├── CompositePass.java │ │ │ │ │ ├── DefaultCompositePass.java │ │ │ │ │ ├── LargeShuffleCompositePass.java │ │ │ │ │ ├── LoopOptimizationCompositePass.java │ │ │ │ │ ├── SkewCompositePass.java │ │ │ │ │ └── TransientResourceCompositePass.java │ │ │ │ └── reshaping │ │ │ │ │ ├── CommonSubexpressionEliminationPass.java │ │ │ │ │ ├── LargeShuffleReshapingPass.java │ │ │ │ │ ├── LoopExtractionPass.java │ │ │ │ │ ├── LoopOptimizations.java │ │ │ │ │ ├── LoopUnrollingPass.java │ │ │ │ │ ├── ReshapingPass.java │ │ │ │ │ ├── SamplingSkewReshapingPass.java │ │ │ │ │ ├── SamplingTaskSizingPass.java │ │ │ │ │ ├── SkewHandlingUtil.java │ │ │ │ │ └── SkewReshapingPass.java │ │ │ └── runtime │ │ │ │ ├── DynamicTaskSizingRuntimePass.java │ │ │ │ ├── Message.java │ │ │ │ ├── RunTimePass.java │ │ │ │ └── SkewRunTimePass.java │ │ │ └── policy │ │ │ ├── BasicPullPolicy.java │ │ │ ├── BasicPushPolicy.java │ │ │ ├── ConditionalLargeShufflePolicy.java │ │ │ ├── DataSkewPolicy.java │ │ │ ├── DefaultPolicy.java │ │ │ ├── DefaultPolicyWithSeparatePass.java │ │ │ ├── DisaggregationPolicy.java │ │ │ ├── DynamicTaskSizingPolicy.java │ │ │ ├── LambdaPolicy.java │ │ │ ├── LargeShufflePolicy.java │ │ │ ├── Policy.java │ │ │ ├── PolicyBuilder.java │ │ │ ├── PolicyImpl.java │ │ │ ├── SamplingLargeShuffleSkewPolicy.java │ │ │ ├── StreamingPolicy.java │ │ │ ├── TransientResourcePolicy.java │ │ │ └── XGBoostPolicy.java │ │ └── test │ │ └── java │ │ └── org │ │ └── apache │ │ └── nemo │ │ └── compiler │ │ └── optimizer │ │ └── policy │ │ ├── PolicyBuilderTest.java │ │ └── PolicyImplTest.java ├── pom.xml └── test │ ├── pom.xml │ └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── nemo │ │ └── compiler │ │ ├── CompilerTestUtil.java │ │ └── optimizer │ │ └── policy │ │ └── TestPolicy.java │ └── test │ └── java │ └── org │ └── apache │ └── nemo │ └── compiler │ ├── backend │ └── nemo │ │ └── DAGConverterTest.java │ ├── frontend │ └── beam │ │ ├── BeamFrontendALSTest.java │ │ ├── BeamFrontendMLRTest.java │ │ └── transform │ │ ├── CombineFnTest.java │ │ ├── CreateViewTransformTest.java │ │ ├── DoFnTransformTest.java │ │ ├── GBKTransformTest.java │ │ └── TestOutputCollector.java │ └── optimizer │ └── pass │ ├── compiletime │ ├── annotating │ │ ├── DefaultEdgeCoderPassTest.java │ │ ├── DefaultParallelismPassTest.java │ │ └── DefaultScheduleGroupPassTest.java │ ├── composite │ │ ├── DisaggregationPassTest.java │ │ ├── LargeShuffleCompositePassTest.java │ │ ├── SkewCompositePassTest.java │ │ └── TransientResourceCompositePassTest.java │ └── reshaping │ │ ├── CommonSubexpressionEliminationPassTest.java │ │ ├── LoopExtractionPassTest.java │ │ ├── LoopFusionPassTest.java │ │ ├── LoopInvariantCodeMotionALSInefficientTest.java │ │ ├── LoopInvariantCodeMotionPassTest.java │ │ ├── LoopUnrollingPassTest.java │ │ └── SamplingTaskSizingPassTest.java │ └── runtime │ ├── DynamicTaskSizingRuntimePassTest.java │ └── SkewRuntimePassTest.java ├── conf ├── pom.xml └── src │ └── main │ └── java │ └── org │ └── apache │ └── nemo │ └── conf │ ├── DataPlaneConf.java │ └── JobConf.java ├── deploy ├── README.md ├── initialize_fresh_ubuntu.sh └── set_hostname.sh ├── examples ├── beam │ ├── pom.xml │ └── src │ │ ├── main │ │ └── java │ │ │ └── org │ │ │ └── apache │ │ │ └── nemo │ │ │ └── examples │ │ │ └── beam │ │ │ ├── AlternatingLeastSquare.java │ │ │ ├── AlternatingLeastSquareInefficient.java │ │ │ ├── BeamWordCount.java │ │ │ ├── Broadcast.java │ │ │ ├── EDGARAvgDocSize.java │ │ │ ├── EDGARDocumentSuccessRate.java │ │ │ ├── EDGARRequestsByCIK.java │ │ │ ├── EDGARTop10BadRefererDocs.java │ │ │ ├── EDGARTop10Documents.java │ │ │ ├── FloatArrayCoder.java │ │ │ ├── GenericSourceSink.java │ │ │ ├── IntArrayCoder.java │ │ │ ├── MinimalWordCount.java │ │ │ ├── MultinomialLogisticRegression.java │ │ │ ├── NemoPipelineOptionsFactory.java │ │ │ ├── NetworkTraceAnalysis.java │ │ │ ├── ObjectCoderForString.java │ │ │ ├── PartitionWordsByLength.java │ │ │ ├── PerKeyMedian.java │ │ │ ├── PerPercentileAverage.java │ │ │ ├── SimpleSumSQL.java │ │ │ ├── WindowedBroadcast.java │ │ │ ├── WindowedWordCount.java │ │ │ ├── WordCount.java │ │ │ ├── WordCountTimeOut1Sec.java │ │ │ └── WriteOneFilePerWindow.java │ │ └── test │ │ └── java │ │ └── org │ │ └── apache │ │ └── nemo │ │ └── examples │ │ └── beam │ │ ├── AlternatingLeastSquareITCase.java │ │ ├── BeamSimpleSumSQLITCase.java │ │ ├── BroadcastITCase.java │ │ ├── EDGARITCase.java │ │ ├── MultinomialLogisticRegressionITCase.java │ │ ├── NetworkTraceAnalysisITCase.java │ │ ├── PartitionWordsByLengthITCase.java │ │ ├── PerKeyMedianITCase.java │ │ ├── PerPercentileAverageITCase.java │ │ ├── TimeoutITCase.java │ │ ├── WindowedBroadcastITCase.java │ │ ├── WindowedWordCountITCase.java │ │ ├── WordCountITCase.java │ │ └── policy │ │ ├── AggressiveSpeculativeCloningPolicyParallelismFive.java │ │ ├── DataSkewPolicyParallelismFive.java │ │ ├── DefaultPolicyParallelismFive.java │ │ ├── DisaggregationPolicyParallelismFive.java │ │ ├── LargeShufflePolicyParallelismFive.java │ │ ├── PolicyTestUtil.java │ │ ├── StreamingPolicyParallelismFive.java │ │ ├── TransientResourcePolicyParallelismFive.java │ │ ├── TransientResourcePolicyParallelismTen.java │ │ └── UpfrontSchedulingPolicyParallelismFive.java ├── nexmark │ ├── pom.xml │ └── src │ │ └── test │ │ └── java │ │ └── org │ │ └── apache │ │ └── nemo │ │ └── examples │ │ └── nexmark │ │ └── NexmarkITCase.java ├── pom.xml ├── resources │ ├── executors │ │ ├── beam_test_executor_resources.json │ │ ├── beam_test_one_executor_resources.json │ │ ├── beam_test_poisoned_executor_resources.json │ │ └── spark_test_executor_resources.json │ ├── inputs │ │ ├── test_input_als │ │ ├── test_input_edgar │ │ ├── test_input_employees.json │ │ ├── test_input_median │ │ ├── test_input_mlr │ │ ├── test_input_network0 │ │ ├── test_input_network1 │ │ ├── test_input_partition │ │ ├── test_input_people.json │ │ ├── test_input_people.txt │ │ ├── test_input_spark_wordcount │ │ ├── test_input_tag │ │ ├── test_input_windowed_wordcount │ │ └── test_input_wordcount │ └── outputs │ │ ├── expected_output_als │ │ ├── expected_output_broadcast │ │ ├── expected_output_median │ │ ├── expected_output_network │ │ ├── expected_output_partition_0 │ │ ├── expected_output_partition_1 │ │ ├── expected_output_partition_2 │ │ ├── expected_output_partition_3 │ │ ├── expected_output_partition_4 │ │ ├── expected_output_partition_5 │ │ ├── expected_output_partition_6 │ │ ├── expected_output_partition_7 │ │ ├── expected_output_partition_8 │ │ ├── expected_output_partition_9 │ │ ├── expected_output_reversed_wordcount_spark │ │ ├── expected_output_simplesql │ │ ├── expected_output_sliding_windowed_wordcount │ │ ├── expected_output_spark_wordcount │ │ ├── expected_output_tag_long │ │ ├── expected_output_tag_short │ │ ├── expected_output_tag_very_long │ │ ├── expected_output_tag_very_very_long │ │ ├── expected_output_windowed_wordcount │ │ ├── expected_output_word_and_line_count │ │ └── expected_output_wordcount └── spark │ ├── pom.xml │ └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── nemo │ │ │ └── examples │ │ │ └── spark │ │ │ ├── JavaMapReduce.java │ │ │ ├── JavaSparkPi.java │ │ │ ├── JavaWordAndLineCount.java │ │ │ ├── JavaWordCount.java │ │ │ └── sql │ │ │ ├── JavaSparkSQLExample.java │ │ │ ├── JavaUserDefinedTypedAggregation.java │ │ │ └── JavaUserDefinedUntypedAggregation.java │ └── scala │ │ └── org │ │ └── apache │ │ └── nemo │ │ └── examples │ │ └── spark │ │ ├── SparkALS.scala │ │ ├── SparkCachingWordCount.scala │ │ ├── SparkPi.scala │ │ └── SparkWordCount.scala │ └── test │ └── java │ └── org │ └── apache │ └── nemo │ └── examples │ └── spark │ ├── MRJava.java │ ├── SparkJava.java │ └── SparkScala.java ├── formatter.xml ├── log4j.properties ├── ml ├── nemo_xgboost_optimization.py └── requirements.txt ├── pom.xml ├── runtime ├── common │ ├── pom.xml │ └── src │ │ ├── main │ │ ├── java │ │ │ └── org │ │ │ │ └── apache │ │ │ │ └── nemo │ │ │ │ └── runtime │ │ │ │ └── common │ │ │ │ ├── NativeChannelImplementationSelector.java │ │ │ │ ├── NettyChannelImplementationSelector.java │ │ │ │ ├── NioChannelImplementationSelector.java │ │ │ │ ├── ReplyFutureMap.java │ │ │ │ ├── RuntimeIdManager.java │ │ │ │ ├── RuntimeTestUtil.java │ │ │ │ ├── exception │ │ │ │ ├── AbsentBlockException.java │ │ │ │ └── PlanAppenderException.java │ │ │ │ ├── message │ │ │ │ ├── ClientRPC.java │ │ │ │ ├── FailedMessageSender.java │ │ │ │ ├── MessageContext.java │ │ │ │ ├── MessageEnvironment.java │ │ │ │ ├── MessageListener.java │ │ │ │ ├── MessageParameters.java │ │ │ │ ├── MessageSender.java │ │ │ │ ├── MessageUtils.java │ │ │ │ ├── PersistentConnectionToMasterMap.java │ │ │ │ ├── grpc │ │ │ │ │ ├── GrpcMessageClient.java │ │ │ │ │ ├── GrpcMessageContext.java │ │ │ │ │ ├── GrpcMessageEnvironment.java │ │ │ │ │ ├── GrpcMessageSender.java │ │ │ │ │ └── GrpcMessageServer.java │ │ │ │ ├── local │ │ │ │ │ ├── LocalMessageContext.java │ │ │ │ │ ├── LocalMessageDispatcher.java │ │ │ │ │ ├── LocalMessageEnvironment.java │ │ │ │ │ └── LocalMessageSender.java │ │ │ │ └── ncs │ │ │ │ │ ├── ControlMessageCodec.java │ │ │ │ │ ├── NcsMessageContext.java │ │ │ │ │ ├── NcsMessageEnvironment.java │ │ │ │ │ └── NcsMessageSender.java │ │ │ │ ├── metric │ │ │ │ ├── DataTransferEvent.java │ │ │ │ ├── Event.java │ │ │ │ ├── JobMetric.java │ │ │ │ ├── LatencyMetric.java │ │ │ │ ├── Metric.java │ │ │ │ ├── MetricUtils.java │ │ │ │ ├── StageMetric.java │ │ │ │ ├── StateMetric.java │ │ │ │ ├── StateTransitionEvent.java │ │ │ │ ├── StreamMetric.java │ │ │ │ └── TaskMetric.java │ │ │ │ ├── plan │ │ │ │ ├── PhysicalPlan.java │ │ │ │ ├── PhysicalPlanGenerator.java │ │ │ │ ├── PlanRewriter.java │ │ │ │ ├── RuntimeEdge.java │ │ │ │ ├── Stage.java │ │ │ │ ├── StageEdge.java │ │ │ │ ├── StagePartitioner.java │ │ │ │ └── Task.java │ │ │ │ └── state │ │ │ │ ├── BlockState.java │ │ │ │ ├── PlanState.java │ │ │ │ ├── StageState.java │ │ │ │ └── TaskState.java │ │ └── proto │ │ │ ├── ControlMessage.proto │ │ │ └── GrcpMessageService.proto │ │ └── test │ │ └── java │ │ └── org │ │ └── apache │ │ └── nemo │ │ └── runtime │ │ └── common │ │ ├── message │ │ └── local │ │ │ └── LocalMessageTest.java │ │ ├── metric │ │ └── MetricUtilsTest.java │ │ └── plan │ │ ├── PhysicalPlanGeneratorTest.java │ │ └── StagePartitionerTest.java ├── driver │ ├── pom.xml │ └── src │ │ └── main │ │ └── java │ │ └── org │ │ └── apache │ │ └── nemo │ │ └── driver │ │ ├── NemoContext.java │ │ ├── NemoDriver.java │ │ ├── RemoteClientMessageLoggingHandler.java │ │ └── UserApplicationRunner.java ├── executor │ ├── pom.xml │ └── src │ │ ├── main │ │ └── java │ │ │ └── org │ │ │ └── apache │ │ │ └── nemo │ │ │ └── runtime │ │ │ └── executor │ │ │ ├── Executor.java │ │ │ ├── MetricManagerWorker.java │ │ │ ├── MetricMessageSender.java │ │ │ ├── TaskStateManager.java │ │ │ ├── TransformContextImpl.java │ │ │ ├── data │ │ │ ├── BlockManagerWorker.java │ │ │ ├── BlockTransferThrottler.java │ │ │ ├── BroadcastManagerWorker.java │ │ │ ├── DataUtil.java │ │ │ ├── DirectByteBufferOutputStream.java │ │ │ ├── FileArea.java │ │ │ ├── LimitedInputStream.java │ │ │ ├── MemoryAllocationException.java │ │ │ ├── MemoryChunk.java │ │ │ ├── MemoryPoolAssigner.java │ │ │ ├── PipeContainer.java │ │ │ ├── PipeManagerWorker.java │ │ │ ├── SerializerManager.java │ │ │ ├── block │ │ │ │ ├── Block.java │ │ │ │ ├── FileBlock.java │ │ │ │ ├── NonSerializedMemoryBlock.java │ │ │ │ └── SerializedMemoryBlock.java │ │ │ ├── metadata │ │ │ │ ├── FileMetadata.java │ │ │ │ ├── LocalFileMetadata.java │ │ │ │ ├── PartitionMetadata.java │ │ │ │ └── RemoteFileMetadata.java │ │ │ ├── partition │ │ │ │ ├── NonSerializedPartition.java │ │ │ │ ├── Partition.java │ │ │ │ └── SerializedPartition.java │ │ │ ├── stores │ │ │ │ ├── AbstractBlockStore.java │ │ │ │ ├── BlockStore.java │ │ │ │ ├── GlusterFileStore.java │ │ │ │ ├── LocalBlockStore.java │ │ │ │ ├── LocalFileStore.java │ │ │ │ ├── MemoryStore.java │ │ │ │ ├── RemoteFileStore.java │ │ │ │ └── SerializedMemoryStore.java │ │ │ └── streamchainer │ │ │ │ ├── CompressionStreamChainer.java │ │ │ │ ├── DecodeStreamChainer.java │ │ │ │ ├── DecompressionStreamChainer.java │ │ │ │ ├── EncodeStreamChainer.java │ │ │ │ └── Serializer.java │ │ │ ├── datatransfer │ │ │ ├── BlockInputReader.java │ │ │ ├── BlockOutputWriter.java │ │ │ ├── DataFetcherOutputCollector.java │ │ │ ├── InputReader.java │ │ │ ├── InputWatermarkManager.java │ │ │ ├── IntermediateDataIOFactory.java │ │ │ ├── MultiInputWatermarkManager.java │ │ │ ├── NemoEventDecoderFactory.java │ │ │ ├── NemoEventEncoderFactory.java │ │ │ ├── NextIntraTaskOperatorInfo.java │ │ │ ├── OperatorVertexOutputCollector.java │ │ │ ├── OperatorWatermarkCollector.java │ │ │ ├── OutputWriter.java │ │ │ ├── PipeInputReader.java │ │ │ ├── PipeOutputWriter.java │ │ │ ├── RunTimeMessageOutputCollector.java │ │ │ ├── SingleInputWatermarkManager.java │ │ │ └── WatermarkWithIndex.java │ │ │ ├── task │ │ │ ├── DataFetcher.java │ │ │ ├── MultiThreadParentTaskDataFetcher.java │ │ │ ├── ParentTaskDataFetcher.java │ │ │ ├── SourceVertexDataFetcher.java │ │ │ ├── TaskExecutor.java │ │ │ └── VertexHarness.java │ │ │ └── transfer │ │ │ ├── ByteInputContext.java │ │ │ ├── ByteOutputContext.java │ │ │ ├── ByteTransfer.java │ │ │ ├── ByteTransferContext.java │ │ │ ├── ByteTransport.java │ │ │ ├── ByteTransportChannelInitializer.java │ │ │ ├── ClosableBlockingQueue.java │ │ │ ├── ContextManager.java │ │ │ ├── ControlFrameEncoder.java │ │ │ ├── DataFrameEncoder.java │ │ │ ├── FrameDecoder.java │ │ │ ├── LocalInputContext.java │ │ │ ├── LocalOutputContext.java │ │ │ ├── LocalTransferContext.java │ │ │ ├── OutputContext.java │ │ │ ├── TransferOutputStream.java │ │ │ └── package-info.java │ │ └── test │ │ └── java │ │ └── org │ │ └── apache │ │ └── nemo │ │ └── runtime │ │ └── executor │ │ ├── MetricFlushTest.java │ │ ├── TestUtil.java │ │ ├── TransformContextImplTest.java │ │ ├── data │ │ ├── BlockStoreTest.java │ │ ├── BlockTest.java │ │ ├── BlockTransferThrottlerTest.java │ │ ├── DirectByteBufferOutputStreamTest.java │ │ ├── MemoryChunkTest.java │ │ └── MemoryPoolAssignerTest.java │ │ ├── datatransfer │ │ ├── DataTransferTest.java │ │ └── InputWatermarkManagerTest.java │ │ ├── task │ │ ├── ParentTaskDataFetcherTest.java │ │ └── TaskExecutorTest.java │ │ └── transfer │ │ └── LocalTransferContextTest.java ├── master │ ├── pom.xml │ └── src │ │ ├── main │ │ └── java │ │ │ └── org │ │ │ └── apache │ │ │ └── nemo │ │ │ └── runtime │ │ │ └── master │ │ │ ├── BlockManagerMaster.java │ │ │ ├── BlockMetadata.java │ │ │ ├── BroadcastManagerMaster.java │ │ │ ├── PipeManagerMaster.java │ │ │ ├── PlanAppender.java │ │ │ ├── PlanStateManager.java │ │ │ ├── RuntimeMaster.java │ │ │ ├── metric │ │ │ ├── MetricBroadcaster.java │ │ │ ├── MetricManagerMaster.java │ │ │ ├── MetricMessageHandler.java │ │ │ └── MetricStore.java │ │ │ ├── resource │ │ │ ├── ContainerManager.java │ │ │ ├── DefaultExecutorRepresenter.java │ │ │ └── ExecutorRepresenter.java │ │ │ ├── scheduler │ │ │ ├── AntiAffinitySchedulingConstraint.java │ │ │ ├── BatchScheduler.java │ │ │ ├── BatchSchedulerUtils.java │ │ │ ├── ContainerTypeAwareSchedulingConstraint.java │ │ │ ├── ExecutorRegistry.java │ │ │ ├── FreeSlotSchedulingConstraint.java │ │ │ ├── LocalitySchedulingConstraint.java │ │ │ ├── MinOccupancyFirstSchedulingPolicy.java │ │ │ ├── NodeShareSchedulingConstraint.java │ │ │ ├── PendingTaskCollectionPointer.java │ │ │ ├── Scheduler.java │ │ │ ├── SchedulingConstraint.java │ │ │ ├── SchedulingConstraintRegistry.java │ │ │ ├── SchedulingPolicy.java │ │ │ ├── SimulatedTaskExecutor.java │ │ │ ├── SimulationScheduler.java │ │ │ ├── StreamingScheduler.java │ │ │ └── TaskDispatcher.java │ │ │ └── servlet │ │ │ ├── AllMetricServlet.java │ │ │ ├── JobMetricServlet.java │ │ │ ├── StageMetricServlet.java │ │ │ ├── TaskMetricServlet.java │ │ │ ├── WebSocketMetricAdapter.java │ │ │ └── WebSocketMetricServlet.java │ │ └── test │ │ └── java │ │ └── org │ │ └── apache │ │ └── nemo │ │ └── runtime │ │ └── master │ │ ├── BlockManagerMasterTest.java │ │ ├── ContainerManagerTest.java │ │ ├── MetricStoreTest.java │ │ ├── PlanStateManagerTest.java │ │ └── scheduler │ │ ├── AntiAffinitySchedulingConstraintTest.java │ │ ├── BatchSchedulerTest.java │ │ ├── ContainerTypeAwareSchedulingConstraintTest.java │ │ ├── FreeSlotSchedulingConstraintTest.java │ │ ├── LocalitySchedulingConstraintTest.java │ │ ├── MinOccupancyFirstSchedulingPolicyTest.java │ │ ├── PendingTaskCollectionPointerTest.java │ │ ├── SchedulerTestUtil.java │ │ ├── SchedulingConstraintnRegistryTest.java │ │ ├── SimulationSchedulerTest.java │ │ ├── StreamingSchedulerTest.java │ │ └── TaskRetryTest.java ├── pom.xml └── test │ ├── pom.xml │ └── src │ └── main │ └── java │ └── org │ └── apache │ └── nemo │ └── runtime │ └── common │ └── plan │ └── TestPlanGenerator.java ├── tools └── network_profiling │ ├── network_clustering.py │ └── network_profiling.py └── webui ├── .editorconfig ├── .eslintrc.js ├── .gitignore ├── README.md ├── assets ├── NemoLogo.png └── constants.js ├── components ├── MainContainer.vue ├── TaskStatistics.vue ├── environment │ └── EnvironmentView.vue ├── executors │ └── ExecutorsView.vue ├── jobs │ ├── JobsView.vue │ └── detail │ │ ├── DAG.vue │ │ ├── DetailTable.vue │ │ ├── JobView.vue │ │ ├── MetricTimeline.vue │ │ └── StageSelect.vue ├── stages │ └── StagesView.vue └── storage │ └── StorageView.vue ├── layouts └── default.vue ├── nuxt.config.js ├── package-lock.json ├── package.json ├── pages └── index.vue ├── plugins ├── element-ui.js ├── event-bus.js ├── vue-affix.js └── vue2vis.js ├── static ├── favicon.ico └── nemo.ico └── test ├── DetailTable.test.js ├── JobView.test.js ├── JobsView.test.js ├── MetricTimeline.test.js ├── StageSelect.test.js └── setup.js /.asf.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, 13 | # software distributed under the License is distributed on an 14 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | # KIND, either express or implied. See the License for the 16 | # specific language governing permissions and limitations 17 | # under the License. 18 | 19 | # Check https://cwiki.apache.org/confluence/display/INFRA/Git+-+.asf.yaml+features 20 | 21 | github: 22 | protected_branches: 23 | master: 24 | required_status_checks: 25 | # strict means "Require branches to be up to date before merging". 26 | strict: true 27 | # contexts are the names of checks that must pass 28 | contexts: 29 | - Build on Java 11 and ubuntu-18.04 30 | - Build on Node 16.x and ubuntu-18.04 31 | 32 | required_pull_request_reviews: 33 | dismiss_stale_reviews: true 34 | required_approving_review_count: 1 35 | 36 | # squash or rebase must be allowed in the repo for this setting to be set to true. 37 | required_linear_history: true 38 | 39 | required_signatures: true 40 | branch_b: 41 | required_signatures: true 42 | del_branch_on_merge: true 43 | -------------------------------------------------------------------------------- /.editorconfig: -------------------------------------------------------------------------------- 1 | root = true 2 | 3 | [*] 4 | charset = utf-8 5 | indent_style = space 6 | indent_size = 2 7 | end_of_line = lf 8 | insert_final_newline = true 9 | trim_trailing_whitespace = true 10 | -------------------------------------------------------------------------------- /.github/pull_request_template.md: -------------------------------------------------------------------------------- 1 | JIRA: [NEMO-###: TITLE](https://issues.apache.org/jira/projects/NEMO/issues/NEMO-###) 2 | 3 | **Major changes:** 4 | - 5 | 6 | **Minor changes to note:** 7 | - 8 | 9 | **Tests for the changes:** 10 | - 11 | 12 | **Other comments:** 13 | - 14 | 15 | Closes #GITHUB_PR_NUMBER 16 | -------------------------------------------------------------------------------- /DISCLAIMER: -------------------------------------------------------------------------------- 1 | Apache Nemo (incubating) is an effort undergoing incubation at The Apache Software Foundation (ASF), 2 | sponsored by the Apache Incubator PMC. Incubation is required of all newly accepted projects 3 | until a further review indicates that the infrastructure, communications, and decision making process have stabilized 4 | in a manner consistent with other successful ASF projects. While incubation status is not necessarily 5 | a reflection of the completeness or stability of the code, 6 | it does indicate that the project has yet to be fully endorsed by the ASF. 7 | -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | Apache Nemo 2 | Copyright 2018 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -------------------------------------------------------------------------------- /bin/generate_javadocs.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | # 20 | # run this by ./bin/generate_javadocs.sh 21 | 22 | echo "mvn javadoc:aggregate" 23 | mvn javadoc:aggregate 24 | -------------------------------------------------------------------------------- /bin/run_beam.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | 20 | VERSION=$(mvn -q \ 21 | -Dexec.executable=echo -Dexec.args='${project.version}' \ 22 | --non-recursive exec:exec) 23 | 24 | java -Dlog4j.configuration=file://`pwd`/log4j.properties -cp examples/beam/target/nemo-examples-beam-${VERSION}-shaded.jar:client/target/nemo-client-${VERSION}-shaded.jar:`$YARN_HOME/bin/yarn classpath` org.apache.nemo.client.JobLauncher "$@" 25 | -------------------------------------------------------------------------------- /bin/run_nexmark.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | 20 | VERSION=$(mvn -q \ 21 | -Dexec.executable=echo -Dexec.args='${project.version}' \ 22 | --non-recursive exec:exec) 23 | 24 | java -Dlog4j.configuration=file://`pwd`/log4j.properties -cp examples/nexmark/target/nemo-examples-nexmark-${VERSION}-shaded.jar:client/target/nemo-client-${VERSION}-shaded.jar:`$YARN_HOME/bin/yarn classpath` org.apache.nemo.client.JobLauncher "$@" 25 | -------------------------------------------------------------------------------- /bin/run_spark.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | 20 | VERSION=$(mvn -q \ 21 | -Dexec.executable=echo -Dexec.args='${project.version}' \ 22 | --non-recursive exec:exec) 23 | 24 | java -Dlog4j.configuration=file://`pwd`/log4j.properties -cp examples/spark/target/nemo-examples-spark-${VERSION}-shaded.jar:`yarn classpath` org.apache.nemo.client.JobLauncher "$@" 25 | -------------------------------------------------------------------------------- /bin/run_webserver.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | 20 | pushd webui 21 | npm install 22 | npm run build 23 | npm start 24 | popd 25 | -------------------------------------------------------------------------------- /bin/sonar_qube.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | 20 | echo "You should already have SonarQube installed and running at localhost:9000" 21 | echo "e.g. OSX: brew install sonarqube && sonar console" 22 | sonar console 23 | mvn clean package sonar:sonar 24 | -------------------------------------------------------------------------------- /bin/xgboost_optimization.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | 20 | echo "You should already have python3 installed" 21 | echo "Usage: ./bin/nemo_xgboost_optimization.sh " 22 | pushd ml 23 | touch results.out 24 | pip3 install -r requirements.txt 25 | python3 nemo_xgboost_optimization.py -t "$@" 26 | popd 27 | -------------------------------------------------------------------------------- /checkstyle.license: -------------------------------------------------------------------------------- 1 | (\s*[/\*|\/\/| 20 | 21 | 4.0.0 22 | 23 | 24 | org.apache.nemo 25 | nemo-project 26 | 0.5-SNAPSHOT 27 | ../ 28 | 29 | 30 | nemo-conf 31 | Nemo Job Configuration 32 | 33 | 34 | 35 | org.apache.reef 36 | reef-common 37 | ${reef.version} 38 | 39 | 40 | 41 | -------------------------------------------------------------------------------- /deploy/set_hostname.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | 20 | echo "./set_hostname.sh remote_hosts_file" 21 | cat $1 | xargs -n 1 -I '{}' ssh '{}' "sudo bash -c 'echo '{}' > /etc/hostname'" 22 | echo "Done! Please check /etc/hostname" 23 | -------------------------------------------------------------------------------- /examples/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 22 | nemo-project 23 | org.apache.nemo 24 | 0.5-SNAPSHOT 25 | 26 | 4.0.0 27 | 28 | nemo-examples 29 | pom 30 | Nemo Examples 31 | 32 | 33 | beam 34 | spark 35 | nexmark 36 | 37 | 38 | 39 | 40 | -------------------------------------------------------------------------------- /examples/resources/executors/beam_test_executor_resources.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "type": "Transient", 4 | "memory_mb": 512, 5 | "capacity": 15 6 | }, 7 | { 8 | "type": "Reserved", 9 | "memory_mb": 512, 10 | "capacity": 15 11 | } 12 | ] 13 | -------------------------------------------------------------------------------- /examples/resources/executors/beam_test_one_executor_resources.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "type": "Transient", 4 | "memory_mb": 512, 5 | "capacity": 15 6 | } 7 | ] 8 | -------------------------------------------------------------------------------- /examples/resources/executors/beam_test_poisoned_executor_resources.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "type": "Transient", 4 | "memory_mb": 512, 5 | "capacity": 15, 6 | "poison_sec": 2 7 | }, 8 | { 9 | "type": "Reserved", 10 | "memory_mb": 512, 11 | "capacity": 15 12 | } 13 | ] 14 | -------------------------------------------------------------------------------- /examples/resources/executors/spark_test_executor_resources.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "type": "Transient", 4 | "memory_mb": 512, 5 | "capacity": 5 6 | }, 7 | { 8 | "type": "Reserved", 9 | "memory_mb": 512, 10 | "capacity": 5 11 | } 12 | ] 13 | -------------------------------------------------------------------------------- /examples/resources/inputs/test_input_employees.json: -------------------------------------------------------------------------------- 1 | {"name":"Michael", "salary":3000} 2 | {"name":"Andy", "salary":4500} 3 | {"name":"Justin", "salary":3500} 4 | {"name":"Berta", "salary":4000} 5 | -------------------------------------------------------------------------------- /examples/resources/inputs/test_input_median: -------------------------------------------------------------------------------- 1 | wonook spark 2 | john flume 3 | gw flink 4 | john flume1 5 | mh beam 6 | john flume2 7 | wonook spark1 8 | jykim apex 9 | john flume3 10 | jykim apex1 11 | mh beam1 12 | jykim apex2 13 | gw flink1 14 | john flume4 15 | -------------------------------------------------------------------------------- /examples/resources/inputs/test_input_network0: -------------------------------------------------------------------------------- 1 | 1 0.0 192.168.0.1 -> 192.168.0.2 Len=32 2 | 2 0.0 192.168.1.1 -> 192.168.0.2 Len=31 3 | 3 0.0 192.168.2.1 -> 192.168.0.2 Len=30 4 | 4 0.0 192.168.3.1 -> 192.168.0.2 Len=29 5 | 5 0.0 192.168.4.1 -> 192.168.0.2 Len=28 6 | 6 0.0 192.168.5.1 -> 192.168.0.3 Len=27 7 | 7 0.0 192.168.5.1 -> 192.168.0.3 Len=26 8 | 8 0.0 192.168.1.1 -> 192.168.0.3 Len=25 9 | -------------------------------------------------------------------------------- /examples/resources/inputs/test_input_network1: -------------------------------------------------------------------------------- 1 | 1 0.0 192.168.0.2 -> 192.168.1.10 Len=32 2 | 2 0.0 192.168.0.2 -> 192.168.1.10 Len=31 3 | 3 0.0 192.168.0.2 -> 192.168.2.10 Len=30 4 | 4 0.0 192.168.0.2 -> 192.168.2.10 Len=29 5 | 5 0.0 192.168.0.2 -> 192.168.3.10 Len=16 6 | 6 0.0 192.168.0.3 -> 192.168.3.10 Len=15 7 | 7 0.0 192.168.0.3 -> 192.168.4.10 Len=14 8 | 8 0.0 192.168.0.3 -> 192.168.4.10 Len=13 9 | -------------------------------------------------------------------------------- /examples/resources/inputs/test_input_partition: -------------------------------------------------------------------------------- 1 | 1 14 23 2 | 2 49 57 3 | 3 23 29 4 | 5 37 37 5 | 6 41 39 6 | 7 57 63 7 | 8 66 69 8 | 9 91 92 9 | 10 93 93 10 | 11 97 98 11 | 12 96 97 12 | 13 82 90 13 | 14 88 91 14 | 15 7 13 15 | 16 11 21 16 | 17 3 9 17 | 18 81 88 18 | 19 99 100 19 | 20 62 65 20 | 21 83 90 21 | 22 26 31 22 | 23 31 34 23 | 24 39 38 24 | 25 44 43 25 | 26 50 61 26 | 27 61 64 27 | 28 67 71 28 | 29 8 17 -------------------------------------------------------------------------------- /examples/resources/inputs/test_input_people.json: -------------------------------------------------------------------------------- 1 | {"name":"Michael"} 2 | {"name":"Andy", "age":30} 3 | {"name":"Justin", "age":19} 4 | -------------------------------------------------------------------------------- /examples/resources/inputs/test_input_people.txt: -------------------------------------------------------------------------------- 1 | Michael, 29 2 | Andy, 30 3 | Justin, 19 4 | -------------------------------------------------------------------------------- /examples/resources/inputs/test_input_spark_wordcount: -------------------------------------------------------------------------------- 1 | banana 2 | ski snowboard 3 | banana ski one two three 4 | bicycle piano piano three 5 | two three ski ski jangho 6 | jangho wonook wonook wonook wonook tennis 7 | john john jy jy jy wonook 8 | ski snowboard tennis 9 | shakespeare sanha girl jy 10 | piano bicycle tennis 11 | banana 12 | -------------------------------------------------------------------------------- /examples/resources/inputs/test_input_tag: -------------------------------------------------------------------------------- 1 | a 2 | to 3 | foo 4 | bar 5 | that 6 | ipsum 7 | dolor 8 | foobar 9 | barbaz 10 | abcdefg 11 | fooipsum 12 | foobarbaz 13 | bazquxfoobarfoobaz 14 | loren 15 | this 16 | foobarbazqux 17 | bazquxfoobar 18 | bazquxfoobarfoobar 19 | qux 20 | ipsumlorem 21 | -------------------------------------------------------------------------------- /examples/resources/inputs/test_input_windowed_wordcount: -------------------------------------------------------------------------------- 1 | wonook m 50!1536907180000 2 | john m 20!1536907181000 3 | gw m 90!1536907182000 4 | john m 20!1536907183000 5 | mh m 30!1536907184000 6 | john m 20!1536907185000 7 | wonook m 50!1536907186000 8 | jykim f 40!1536907187000 9 | john m 20!1536907188000 10 | jykim f 20!1536907189000 11 | mh m 70!1536907190000 12 | jykim f 40!1536907191000 13 | gw m 10!1536907192000 14 | john m 20!1536907193000 15 | john m 30!1536907194000 16 | -------------------------------------------------------------------------------- /examples/resources/inputs/test_input_wordcount: -------------------------------------------------------------------------------- 1 | wonook m 50 2 | john m 20 3 | gw m 90 4 | john m 20 5 | mh m 30 6 | john m 20 7 | wonook m 50 8 | jykim f 40 9 | john m 20 10 | jykim f 20 11 | mh m 70 12 | jykim f 40 13 | gw m 10 14 | john m 20 15 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_median: -------------------------------------------------------------------------------- 1 | gw, flink1 2 | john, flume2 3 | jykim, apex1 4 | mh, beam1 5 | wonook, spark1 6 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_network: -------------------------------------------------------------------------------- 1 | 192.168.0.2,2,7 2 | 192.168.0.3,1,1 3 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_partition_0: -------------------------------------------------------------------------------- 1 | 3 13.0 2 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_partition_1: -------------------------------------------------------------------------------- 1 | 2 22.0 2 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_partition_2: -------------------------------------------------------------------------------- 1 | 2 30.0 2 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_partition_3: -------------------------------------------------------------------------------- 1 | 3 36.333333333333336 2 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_partition_4: -------------------------------------------------------------------------------- 1 | 3 46.333333333333336 2 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_partition_5: -------------------------------------------------------------------------------- 1 | 2 62.0 2 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_partition_6: -------------------------------------------------------------------------------- 1 | 4 67.25 2 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_partition_7: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/incubator-nemo/d384c79cb2e58481b0338cfc436c259b57dc88f9/examples/resources/outputs/expected_output_partition_7 -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_partition_8: -------------------------------------------------------------------------------- 1 | 4 89.75 2 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_partition_9: -------------------------------------------------------------------------------- 1 | 5 96.0 2 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_reversed_wordcount_spark: -------------------------------------------------------------------------------- 1 | 1: one, girl, shakespeare, sanha 2 | 2: bicycle, two, john, jangho, snowboard 3 | 3: banana, three, tennis, piano 4 | 4: jy 5 | 5: ski, wonook 6 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_simplesql: -------------------------------------------------------------------------------- 1 | row is 44.0 2 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_sliding_windowed_wordcount: -------------------------------------------------------------------------------- 1 | gw#m: 10 2 | gw#m: 10 3 | gw#m: 90 4 | gw#m: 90 5 | john#m: 40 6 | john#m: 50 7 | john#m: 80 8 | john#m: 90 9 | jykim#f: 100 10 | jykim#f: 40 11 | jykim#f: 60 12 | mh#m: 30 13 | mh#m: 30 14 | mh#m: 70 15 | mh#m: 70 16 | wonook#m: 100 17 | wonook#m: 50 18 | wonook#m: 50 19 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_spark_wordcount: -------------------------------------------------------------------------------- 1 | banana: 3 2 | bicycle: 2 3 | one: 1 4 | girl: 1 5 | two: 2 6 | three: 3 7 | tennis: 3 8 | jy: 4 9 | ski: 5 10 | piano: 3 11 | wonook: 5 12 | shakespeare: 1 13 | john: 2 14 | jangho: 2 15 | sanha: 1 16 | snowboard: 2 17 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_tag_long: -------------------------------------------------------------------------------- 1 | 6: foobar 2 | 6: barbaz 3 | 4 | 7: abcdefg 5 | 6 | 8: fooipsum 7 | 8 | 9: foobarbaz 9 | 10 | 10: ipsumlorem 11 | 12 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_tag_short: -------------------------------------------------------------------------------- 1 | 1: a 2 | 3 | 2: to 4 | 5 | 3: foo 6 | 3: bar 7 | 3: qux 8 | 9 | 4: that 10 | 4: this 11 | 12 | 5: ipsum 13 | 5: dolor 14 | 5: loren 15 | 16 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_tag_very_long: -------------------------------------------------------------------------------- 1 | foobarbazqux 2 | bazquxfoobar 3 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_tag_very_very_long: -------------------------------------------------------------------------------- 1 | bazquxfoobarfoobaz 2 | bazquxfoobarfoobar 3 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_windowed_wordcount: -------------------------------------------------------------------------------- 1 | gw#m: 10 2 | gw#m: 90 3 | john#m: 40 4 | john#m: 40 5 | john#m: 50 6 | jykim#f: 40 7 | jykim#f: 60 8 | mh#m: 30 9 | mh#m: 70 10 | wonook#m: 50 11 | wonook#m: 50 12 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_word_and_line_count: -------------------------------------------------------------------------------- 1 | line count: 11 2 | 3 | banana: 3 4 | bicycle: 2 5 | one: 1 6 | girl: 1 7 | two: 2 8 | three: 3 9 | tennis: 3 10 | jy: 4 11 | ski: 5 12 | piano: 3 13 | wonook: 5 14 | shakespeare: 1 15 | john: 2 16 | jangho: 2 17 | sanha: 1 18 | snowboard: 2 19 | -------------------------------------------------------------------------------- /examples/resources/outputs/expected_output_wordcount: -------------------------------------------------------------------------------- 1 | mh#m: 100 2 | jykim#f: 100 3 | gw#m: 100 4 | wonook#m: 100 5 | john#m: 100 6 | -------------------------------------------------------------------------------- /log4j.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, 13 | # software distributed under the License is distributed on an 14 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | # KIND, either express or implied. See the License for the 16 | # specific language governing permissions and limitations 17 | # under the License. 18 | # 19 | log4j.rootLogger=INFO, STDOUT 20 | 21 | log4j.logger.org.apache.nemo.runtime.master.metric.MetricManagerMaster=INFO, METRIC 22 | log4j.additivity.org.apache.nemo.runtime.master.metric.MetricManagerMaster=false 23 | 24 | log4j.appender.STDOUT=org.apache.log4j.ConsoleAppender 25 | log4j.appender.STDOUT.layout=org.apache.log4j.PatternLayout 26 | log4j.appender.STDOUT.layout.ConversionPattern=%5p %d{MM-dd HH:mm:ss,SSS} %C{1}:%L [%t] - %m%n 27 | 28 | log4j.appender.METRIC=org.apache.log4j.FileAppender 29 | log4j.appender.METRIC.File=metric.log 30 | log4j.appender.METRIC.layout=org.apache.log4j.PatternLayout 31 | log4j.appender.METRIC.layout.ConversionPattern=%5p %d{MM-dd HH:mm:ss,SSS} %C{1}:%L [%t] - %m%n 32 | 33 | 34 | # Sample appender 35 | #log4j.appender.file=org.apache.log4j.RollingFileAppender 36 | #log4j.appender.file.File=.\\logging.log 37 | #log4j.appender.file.layout=org.apache.log4j.PatternLayout 38 | #log4j.appender.file.layout.ConversionPattern=%5p %d{MM-dd HH:mm:ss,SSS} %C{1}:%L [%t] - %m%n 39 | -------------------------------------------------------------------------------- /ml/requirements.txt: -------------------------------------------------------------------------------- 1 | graphviz==0.20 2 | matplotlib==3.5.2 3 | numpy==1.23.1 4 | psycopg2==2.9.3 5 | psycopg2-binary==2.9.3 6 | pygraphviz==1.9 7 | xgboost==1.6.1 8 | pandas==1.4.3 9 | scikit-learn==1.1.1 10 | -------------------------------------------------------------------------------- /runtime/common/src/main/java/org/apache/nemo/runtime/common/exception/AbsentBlockException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.common.exception; 20 | 21 | import org.apache.nemo.runtime.common.state.BlockState; 22 | 23 | /** 24 | * An exception which represents the requested block is neither AVAILABLE nor IN_PROGRESS. 25 | */ 26 | public final class AbsentBlockException extends Exception { 27 | private final String blockId; 28 | private final BlockState.State state; 29 | 30 | /** 31 | * @param blockId id of the block 32 | * @param state state of the block 33 | */ 34 | public AbsentBlockException(final String blockId, final BlockState.State state) { 35 | this.blockId = blockId; 36 | this.state = state; 37 | } 38 | 39 | /** 40 | * @return id of the block 41 | */ 42 | public String getBlockId() { 43 | return blockId; 44 | } 45 | 46 | /** 47 | * @return state of the block 48 | */ 49 | public BlockState.State getState() { 50 | return state; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /runtime/common/src/main/java/org/apache/nemo/runtime/common/exception/PlanAppenderException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.common.exception; 20 | 21 | /** 22 | * An exception which represents exception during appending plans. 23 | */ 24 | public final class PlanAppenderException extends RuntimeException { 25 | 26 | /** 27 | * Constructor with throwable. 28 | * 29 | * @param throwable the throwable to throw. 30 | */ 31 | public PlanAppenderException(final Throwable throwable) { 32 | super(throwable); 33 | } 34 | 35 | /** 36 | * Constructor with String. 37 | * 38 | * @param message the exception message. 39 | */ 40 | public PlanAppenderException(final String message) { 41 | super(message); 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /runtime/common/src/main/java/org/apache/nemo/runtime/common/message/FailedMessageSender.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.common.message; 20 | 21 | import org.apache.nemo.runtime.common.comm.ControlMessage; 22 | 23 | import java.util.concurrent.CompletableFuture; 24 | 25 | /** 26 | * A message sender that failed. 27 | */ 28 | public final class FailedMessageSender implements MessageSender { 29 | @Override 30 | public void send(final ControlMessage.Message message) { 31 | // Do nothing. 32 | } 33 | 34 | @Override 35 | public CompletableFuture request(final ControlMessage.Message message) { 36 | final CompletableFuture failed = new CompletableFuture<>(); 37 | failed.completeExceptionally(new Throwable("Failed Message Sender")); 38 | return failed; 39 | } 40 | 41 | @Override 42 | public void close() throws Exception { 43 | // Do nothing. 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /runtime/common/src/main/java/org/apache/nemo/runtime/common/message/MessageContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.common.message; 20 | 21 | /** 22 | * This class sends a reply message from {@link MessageListener}. 23 | */ 24 | public interface MessageContext { 25 | 26 | /** 27 | * Send back a reply message. 28 | * 29 | * @param replyMessage a reply message 30 | * @param type of the reply message 31 | */ 32 | void reply(U replyMessage); 33 | 34 | } 35 | -------------------------------------------------------------------------------- /runtime/common/src/main/java/org/apache/nemo/runtime/common/message/MessageListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.common.message; 20 | 21 | 22 | /** 23 | * Handles messages from {@link MessageSender}. Multiple MessageListeners can be setup using {@link MessageEnvironment} 24 | * while they are identified by their unique message type ids. 25 | * 26 | * @param message type 27 | */ 28 | public interface MessageListener { 29 | 30 | /** 31 | * Called back when a message is received. 32 | * 33 | * @param message a message 34 | */ 35 | void onMessage(T message); 36 | 37 | /** 38 | * Called back when a message is received, and return a response using {@link MessageContext}. 39 | * 40 | * @param message a message 41 | * @param messageContext a message context 42 | */ 43 | void onMessageWithContext(T message, MessageContext messageContext); 44 | 45 | } 46 | -------------------------------------------------------------------------------- /runtime/common/src/main/java/org/apache/nemo/runtime/common/message/MessageParameters.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.common.message; 20 | 21 | import org.apache.reef.tang.annotations.Name; 22 | import org.apache.reef.tang.annotations.NamedParameter; 23 | 24 | /** 25 | * Parameters for message components. 26 | */ 27 | public class MessageParameters { 28 | 29 | /** 30 | * Id of the sender. 31 | */ 32 | @NamedParameter 33 | public static final class SenderId implements Name { 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /runtime/common/src/main/java/org/apache/nemo/runtime/common/message/ncs/ControlMessageCodec.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.common.message.ncs; 20 | 21 | import com.google.protobuf.InvalidProtocolBufferException; 22 | import org.apache.nemo.runtime.common.comm.ControlMessage; 23 | import org.apache.reef.io.serialization.Codec; 24 | 25 | /** 26 | * Codec for ControlMessage. 27 | */ 28 | final class ControlMessageCodec implements Codec, 29 | org.apache.reef.wake.remote.Codec { 30 | 31 | ControlMessageCodec() { 32 | } 33 | 34 | @Override 35 | public byte[] encode(final ControlMessage.Message obj) { 36 | return obj.toByteArray(); 37 | } 38 | 39 | @Override 40 | public ControlMessage.Message decode(final byte[] buf) { 41 | try { 42 | return ControlMessage.Message.parseFrom(buf); 43 | } catch (final InvalidProtocolBufferException e) { 44 | throw new RuntimeException(e); 45 | } 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /runtime/common/src/main/java/org/apache/nemo/runtime/common/metric/Event.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.common.metric; 20 | 21 | import java.io.Serializable; 22 | 23 | /** 24 | * Class for all generic event that contains timestamp at the moment. 25 | */ 26 | public class Event implements Serializable { 27 | private long timestamp; 28 | 29 | /** 30 | * Constructor. 31 | * 32 | * @param timestamp timestamp in millisecond. 33 | */ 34 | public Event(final long timestamp) { 35 | this.timestamp = timestamp; 36 | } 37 | 38 | /** 39 | * Get timestamp. 40 | * 41 | * @return timestamp. 42 | */ 43 | public final long getTimestamp() { 44 | return timestamp; 45 | } 46 | 47 | /** 48 | * Set timestamp. 49 | * 50 | * @param timestamp timestamp in millisecond. 51 | */ 52 | public final void setTimestamp(final long timestamp) { 53 | this.timestamp = timestamp; 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /runtime/common/src/main/java/org/apache/nemo/runtime/common/metric/Metric.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.common.metric; 20 | 21 | /** 22 | * Interface for all metrics. 23 | */ 24 | public interface Metric { 25 | /** 26 | * Get its unique id. 27 | * 28 | * @return an unique id 29 | */ 30 | String getId(); 31 | 32 | /** 33 | * Process metric message from evaluators. 34 | * 35 | * @param metricField field name of the metric. 36 | * @param metricValue byte array of serialized data value. 37 | * @return true if the metric was changed or false if not. 38 | */ 39 | boolean processMetricMessage(String metricField, byte[] metricValue); 40 | } 41 | -------------------------------------------------------------------------------- /runtime/common/src/main/java/org/apache/nemo/runtime/common/metric/StateMetric.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.common.metric; 20 | 21 | import java.io.Serializable; 22 | import java.util.List; 23 | 24 | /** 25 | * Interface for metric which contians its state. 26 | * 27 | * @param class of state of the metric. 28 | */ 29 | public interface StateMetric extends Metric { 30 | /** 31 | * Get its list of {@link StateTransitionEvent}. 32 | * 33 | * @return list of events. 34 | */ 35 | List> getStateTransitionEvents(); 36 | 37 | /** 38 | * Add a {@link StateTransitionEvent} to the metric. 39 | * 40 | * @param prevState previous state. 41 | * @param newState new state. 42 | */ 43 | void addEvent(T prevState, T newState); 44 | } 45 | -------------------------------------------------------------------------------- /runtime/common/src/main/java/org/apache/nemo/runtime/common/metric/StateTransitionEvent.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.common.metric; 20 | 21 | import java.io.Serializable; 22 | 23 | /** 24 | * Event of state transition. It contains timestamp and the state transition. 25 | * 26 | * @param class of state for the metric. 27 | */ 28 | public final class StateTransitionEvent extends Event { 29 | private T prevState; 30 | private T newState; 31 | 32 | public StateTransitionEvent(final long timestamp, final T prevState, final T newState) { 33 | super(timestamp); 34 | this.prevState = prevState; 35 | this.newState = newState; 36 | } 37 | 38 | /** 39 | * Get previous state. 40 | * 41 | * @return previous state. 42 | */ 43 | public T getPrevState() { 44 | return prevState; 45 | } 46 | 47 | /** 48 | * Get new state. 49 | * 50 | * @return new state. 51 | */ 52 | public T getNewState() { 53 | return newState; 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /runtime/common/src/main/java/org/apache/nemo/runtime/common/plan/PlanRewriter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.common.plan; 20 | 21 | import java.util.Set; 22 | 23 | /** 24 | * PhysicalPlan rewriter. 25 | */ 26 | public interface PlanRewriter { 27 | /** 28 | * @param messageId of the rewrite. 29 | * @return physical plan. 30 | */ 31 | PhysicalPlan rewrite(int messageId); 32 | 33 | /** 34 | * @param messageId of the rewrite. 35 | * @param data to accumulate. 36 | */ 37 | void accumulate(int messageId, Set targetEdges, Object data); 38 | } 39 | -------------------------------------------------------------------------------- /runtime/common/src/main/proto/GrcpMessageService.proto: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | syntax = "proto2"; 20 | 21 | package protobuf; 22 | 23 | option java_package = "org.apache.nemo.runtime.common.comm"; 24 | option java_outer_classname = "GrpcMessageService"; 25 | 26 | import "ControlMessage.proto"; 27 | 28 | // A grpc service for implementing org.apache.nemo.runtime.common.message components using grpc 29 | service MessageService { 30 | 31 | // See org.apache.nemo.runtime.common.message.MessageSender#send for more details 32 | rpc send(Message) returns (Void) {} 33 | 34 | // See org.apache.nemo.runtime.common.message.MessageSender#request for more details 35 | rpc request(Message) returns (Message) {} 36 | 37 | } 38 | 39 | message Void { 40 | } 41 | -------------------------------------------------------------------------------- /runtime/executor/src/main/java/org/apache/nemo/runtime/executor/MetricMessageSender.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.executor; 20 | 21 | import org.apache.reef.tang.annotations.DefaultImplementation; 22 | 23 | /** 24 | * Interface for metric sender. 25 | */ 26 | @DefaultImplementation(MetricManagerWorker.class) 27 | public interface MetricMessageSender extends AutoCloseable { 28 | 29 | /** 30 | * Send metric to master. 31 | * 32 | * @param metricType type of the metric 33 | * @param metricId id of the metric 34 | * @param metricField field of the metric 35 | * @param metricValue value of the metric which is serialized 36 | */ 37 | void send(String metricType, String metricId, String metricField, byte[] metricValue); 38 | 39 | /** 40 | * Flush all metric inside of the queue. 41 | */ 42 | void flush(); 43 | 44 | /** 45 | * Flush the metric queue and close the metric dispatch. 46 | */ 47 | void close(); 48 | } 49 | -------------------------------------------------------------------------------- /runtime/executor/src/main/java/org/apache/nemo/runtime/executor/data/MemoryAllocationException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.nemo.runtime.executor.data; 21 | 22 | /** 23 | * An exception to be thrown when a memory allocation operation is not successful. 24 | * Cases where allocation is not successful are when the allocated memory exceeds the 25 | * amount specified by the job configuration or 26 | */ 27 | public class MemoryAllocationException extends Exception { 28 | 29 | public MemoryAllocationException(final String message) { 30 | super(message); 31 | } 32 | 33 | } 34 | -------------------------------------------------------------------------------- /runtime/executor/src/main/java/org/apache/nemo/runtime/executor/data/stores/RemoteFileStore.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.executor.data.stores; 20 | 21 | import org.apache.reef.tang.annotations.DefaultImplementation; 22 | 23 | /** 24 | * Interface for remote block stores (e.g., GlusterFS, ...). 25 | */ 26 | @DefaultImplementation(GlusterFileStore.class) 27 | public interface RemoteFileStore extends BlockStore { 28 | } 29 | -------------------------------------------------------------------------------- /runtime/executor/src/main/java/org/apache/nemo/runtime/executor/data/streamchainer/DecodeStreamChainer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.executor.data.streamchainer; 20 | 21 | import java.io.IOException; 22 | import java.io.InputStream; 23 | 24 | /** 25 | * A {@link DecodeStreamChainer} object indicates each stream manipulation strategy. 26 | * Stream can be chained by {@link DecodeStreamChainer} multiple times. 27 | */ 28 | public interface DecodeStreamChainer { 29 | 30 | /** 31 | * Chain {@link InputStream} and returns chained {@link InputStream}. 32 | * 33 | * @param in the stream which will be chained. 34 | * @return chained {@link InputStream}. 35 | * @throws IOException if fail to chain the stream. 36 | */ 37 | InputStream chainInput(InputStream in) throws IOException; 38 | } 39 | -------------------------------------------------------------------------------- /runtime/executor/src/main/java/org/apache/nemo/runtime/executor/data/streamchainer/EncodeStreamChainer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.executor.data.streamchainer; 20 | 21 | import java.io.IOException; 22 | import java.io.OutputStream; 23 | 24 | /** 25 | * A {@link EncodeStreamChainer} object indicates each stream manipulation strategy. 26 | * Stream can be chained by {@link EncodeStreamChainer} multiple times. 27 | */ 28 | public interface EncodeStreamChainer { 29 | 30 | /** 31 | * Chain {@link OutputStream} and returns chained {@link OutputStream}. 32 | * 33 | * @param out the stream which will be chained. 34 | * @return chained {@link OutputStream}. 35 | * @throws IOException if fail to chain the stream. 36 | */ 37 | OutputStream chainOutput(OutputStream out) throws IOException; 38 | } 39 | -------------------------------------------------------------------------------- /runtime/executor/src/main/java/org/apache/nemo/runtime/executor/transfer/OutputContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.executor.transfer; 20 | 21 | import java.io.IOException; 22 | 23 | /** 24 | * Represents the output context during the data transfer between two tasks. 25 | */ 26 | public interface OutputContext extends AutoCloseable { 27 | 28 | /** 29 | * Creates a new output stream to which the sender sends its data. 30 | * @return output stream to which the sender sends its data. 31 | * @throws IOException if a channel error occurs, or the context has already been closed. 32 | */ 33 | TransferOutputStream newOutputStream() throws IOException; 34 | 35 | /** 36 | * Closes this output context. 37 | * @throws IOException if any exception has occurred. For more information, see 38 | * {@link org.apache.nemo.runtime.executor.transfer.ByteOutputContext.ByteOutputStream#close}. 39 | */ 40 | void close() throws IOException; 41 | } 42 | -------------------------------------------------------------------------------- /runtime/executor/src/main/java/org/apache/nemo/runtime/executor/transfer/TransferOutputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.executor.transfer; 20 | 21 | import org.apache.nemo.runtime.executor.data.streamchainer.Serializer; 22 | import java.io.IOException; 23 | 24 | /** 25 | * Represents the output stream to which the sender sends its data during the data transfer. 26 | */ 27 | public interface TransferOutputStream extends AutoCloseable { 28 | /** 29 | * Write an element into the output stream. 30 | * @param element element to be sent 31 | * @param serializer serializer of {@code element} 32 | */ 33 | void writeElement(Object element, Serializer serializer); 34 | 35 | /** 36 | * Closes this output stream. 37 | * @throws IOException if any exception has occurred. For more information, see {@link ByteOutputContext#close}. 38 | */ 39 | void close() throws IOException; 40 | } 41 | -------------------------------------------------------------------------------- /runtime/executor/src/main/java/org/apache/nemo/runtime/executor/transfer/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | /** 20 | * Responsible for transferring block content from one executor to another. 21 | */ 22 | package org.apache.nemo.runtime.executor.transfer; 23 | -------------------------------------------------------------------------------- /runtime/executor/src/test/java/org/apache/nemo/runtime/executor/TestUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.executor; 20 | 21 | import org.apache.nemo.runtime.common.RuntimeIdManager; 22 | import org.apache.nemo.runtime.common.plan.Stage; 23 | 24 | import java.util.ArrayList; 25 | import java.util.List; 26 | 27 | public final class TestUtil { 28 | public static List generateTaskIds(final Stage stage) { 29 | final List result = new ArrayList<>(); 30 | final int first_attempt = 0; 31 | for (final int taskIndex : stage.getTaskIndices()) { 32 | result.add(RuntimeIdManager.generateTaskId(stage.getId(), taskIndex, first_attempt)); 33 | } 34 | return result; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /runtime/master/src/main/java/org/apache/nemo/runtime/master/metric/MetricMessageHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.master.metric; 20 | 21 | import org.apache.reef.tang.annotations.DefaultImplementation; 22 | 23 | /** 24 | * Metric message handler. 25 | */ 26 | @DefaultImplementation(MetricManagerMaster.class) 27 | public interface MetricMessageHandler { 28 | 29 | /** 30 | * Handle the received metric message. 31 | * 32 | * @param metricType a given type for the metric (ex. TaskMetric). 33 | * @param metricId id of the metric. 34 | * @param metricField field name of the metric. 35 | * @param metricValue serialized metric data value. 36 | */ 37 | void onMetricMessageReceived(String metricType, String metricId, 38 | String metricField, byte[] metricValue); 39 | 40 | /** 41 | * Cleans up and terminates this handler. 42 | */ 43 | void terminate(); 44 | } 45 | -------------------------------------------------------------------------------- /runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/SchedulingConstraint.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.master.scheduler; 20 | 21 | import org.apache.nemo.runtime.common.plan.Task; 22 | import org.apache.nemo.runtime.master.resource.ExecutorRepresenter; 23 | import org.apache.reef.annotations.audience.DriverSide; 24 | 25 | import javax.annotation.concurrent.ThreadSafe; 26 | 27 | /** 28 | * Functions to test schedulability with a pair of an executor and a task. 29 | */ 30 | @DriverSide 31 | @ThreadSafe 32 | @FunctionalInterface 33 | public interface SchedulingConstraint { 34 | boolean testSchedulability(ExecutorRepresenter executor, Task task); 35 | } 36 | -------------------------------------------------------------------------------- /runtime/master/src/main/java/org/apache/nemo/runtime/master/servlet/AllMetricServlet.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.master.servlet; 20 | 21 | import org.apache.nemo.runtime.master.metric.MetricStore; 22 | 23 | import javax.servlet.http.HttpServlet; 24 | import javax.servlet.http.HttpServletRequest; 25 | import javax.servlet.http.HttpServletResponse; 26 | import java.io.IOException; 27 | 28 | /** 29 | * Servlet which handles total metric request. 30 | */ 31 | public final class AllMetricServlet extends HttpServlet { 32 | 33 | @Override 34 | protected void doGet(final HttpServletRequest request, final HttpServletResponse response) 35 | throws IOException { 36 | final MetricStore metricStore = MetricStore.getStore(); 37 | response.setContentType("application/json"); 38 | response.setStatus(HttpServletResponse.SC_OK); 39 | response.getWriter().println(metricStore.dumpAllMetricToJson()); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /runtime/master/src/main/java/org/apache/nemo/runtime/master/servlet/JobMetricServlet.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.master.servlet; 20 | 21 | import org.apache.nemo.runtime.common.metric.JobMetric; 22 | import org.apache.nemo.runtime.master.metric.MetricStore; 23 | 24 | import javax.servlet.http.HttpServlet; 25 | import javax.servlet.http.HttpServletRequest; 26 | import javax.servlet.http.HttpServletResponse; 27 | import java.io.IOException; 28 | 29 | /** 30 | * Servlet which handles {@link JobMetric} metric request. 31 | */ 32 | public final class JobMetricServlet extends HttpServlet { 33 | 34 | @Override 35 | protected void doGet(final HttpServletRequest request, final HttpServletResponse response) 36 | throws IOException { 37 | final MetricStore metricStore = MetricStore.getStore(); 38 | response.setContentType("application/json"); 39 | response.setStatus(HttpServletResponse.SC_OK); 40 | response.getWriter().println(metricStore.dumpMetricToJson(JobMetric.class)); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /runtime/master/src/main/java/org/apache/nemo/runtime/master/servlet/StageMetricServlet.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.master.servlet; 20 | 21 | import org.apache.nemo.runtime.common.metric.StageMetric; 22 | import org.apache.nemo.runtime.master.metric.MetricStore; 23 | 24 | import javax.servlet.http.HttpServlet; 25 | import javax.servlet.http.HttpServletRequest; 26 | import javax.servlet.http.HttpServletResponse; 27 | import java.io.IOException; 28 | 29 | /** 30 | * Servlet which handles {@link StageMetric} metric request. 31 | */ 32 | public final class StageMetricServlet extends HttpServlet { 33 | 34 | @Override 35 | protected void doGet(final HttpServletRequest request, final HttpServletResponse response) 36 | throws IOException { 37 | final MetricStore metricStore = MetricStore.getStore(); 38 | response.setContentType("application/json"); 39 | response.setStatus(HttpServletResponse.SC_OK); 40 | response.getWriter().println(metricStore.dumpMetricToJson(StageMetric.class)); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /runtime/master/src/main/java/org/apache/nemo/runtime/master/servlet/TaskMetricServlet.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.master.servlet; 20 | 21 | import org.apache.nemo.runtime.common.metric.TaskMetric; 22 | import org.apache.nemo.runtime.master.metric.MetricStore; 23 | 24 | import javax.servlet.http.HttpServlet; 25 | import javax.servlet.http.HttpServletRequest; 26 | import javax.servlet.http.HttpServletResponse; 27 | import java.io.IOException; 28 | 29 | /** 30 | * Servlet which handles {@link TaskMetric} metric request. 31 | */ 32 | public final class TaskMetricServlet extends HttpServlet { 33 | 34 | @Override 35 | protected void doGet(final HttpServletRequest request, final HttpServletResponse response) 36 | throws IOException { 37 | final MetricStore metricStore = MetricStore.getStore(); 38 | response.setContentType("application/json"); 39 | response.setStatus(HttpServletResponse.SC_OK); 40 | response.getWriter().println(metricStore.dumpMetricToJson(TaskMetric.class)); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /runtime/master/src/main/java/org/apache/nemo/runtime/master/servlet/WebSocketMetricServlet.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.nemo.runtime.master.servlet; 20 | 21 | import org.eclipse.jetty.websocket.servlet.WebSocketServlet; 22 | import org.eclipse.jetty.websocket.servlet.WebSocketServletFactory; 23 | 24 | /** 25 | * Servlet which handles WebSocket HTTP request. 26 | */ 27 | public class WebSocketMetricServlet extends WebSocketServlet { 28 | 29 | @Override 30 | public final void configure(final WebSocketServletFactory factory) { 31 | // registers WebSocket adapter 32 | factory.register(WebSocketMetricAdapter.class); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /runtime/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 22 | nemo-project 23 | org.apache.nemo 24 | 0.5-SNAPSHOT 25 | ../ 26 | 27 | 4.0.0 28 | 29 | nemo-runtime 30 | pom 31 | Nemo Runtime 32 | 33 | 34 | common 35 | driver 36 | executor 37 | master 38 | test 39 | 40 | 41 | 42 | -------------------------------------------------------------------------------- /runtime/test/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 4.0.0 22 | 23 | 24 | org.apache.nemo 25 | nemo-runtime 26 | 0.5-SNAPSHOT 27 | ../ 28 | 29 | 30 | nemo-runtime-test 31 | Nemo Runtime Test 32 | 33 | 34 | 35 | org.apache.nemo 36 | nemo-runtime-common 37 | ${project.version} 38 | 39 | 40 | org.apache.nemo 41 | nemo-compiler-optimizer 42 | ${project.version} 43 | 44 | 45 | 46 | -------------------------------------------------------------------------------- /webui/.editorconfig: -------------------------------------------------------------------------------- 1 | # editorconfig.org 2 | root = true 3 | 4 | [*] 5 | indent_size = 2 6 | indent_style = space 7 | end_of_line = lf 8 | charset = utf-8 9 | trim_trailing_whitespace = true 10 | insert_final_newline = true 11 | 12 | [*.md] 13 | trim_trailing_whitespace = false 14 | -------------------------------------------------------------------------------- /webui/.eslintrc.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | module.exports = { 20 | root: true, 21 | env: { 22 | browser: true, 23 | node: true 24 | }, 25 | parserOptions: { 26 | parser: 'babel-eslint' 27 | }, 28 | extends: [ 29 | // https://github.com/vuejs/eslint-plugin-vue#priority-a-essential-error-prevention 30 | // consider switching to `plugin:vue/strongly-recommended` or `plugin:vue/recommended` for stricter rules. 31 | 'plugin:vue/essential' 32 | ], 33 | // required to lint *.vue files 34 | plugins: [ 35 | 'vue' 36 | ], 37 | // add your custom rules here 38 | rules: {} 39 | } 40 | -------------------------------------------------------------------------------- /webui/.gitignore: -------------------------------------------------------------------------------- 1 | # dependencies 2 | node_modules 3 | 4 | # logs 5 | npm-debug.log 6 | 7 | # Nuxt build 8 | .nuxt 9 | 10 | # Nuxt generate 11 | dist 12 | 13 | /assets/*.json 14 | -------------------------------------------------------------------------------- /webui/README.md: -------------------------------------------------------------------------------- 1 | # Nemo Web Visualizer 2 | 3 | > Nemo web visualizer 4 | 5 | ## Build Setup 6 | 7 | ``` bash 8 | # install dependencies 9 | $ npm install # Or yarn install 10 | 11 | # serve with hot reload at localhost:3333 12 | $ npm run dev 13 | 14 | # build for production and launch server 15 | $ npm run build 16 | $ npm start 17 | 18 | # generate static project 19 | $ npm run generate 20 | 21 | # run unit test 22 | $ npm test 23 | ``` 24 | 25 | For more information about the commands, refer to [this link](https://nuxtjs.org/guide/commands) 26 | 27 | ## References 28 | 29 | - [VueJS](https://vuejs.org) 30 | - [NuxtJS](https://nuxtjs.org) 31 | - [ElementUI](https://element.eleme.io/#/en-US) 32 | - [Vis](http://visjs.org/) 33 | -------------------------------------------------------------------------------- /webui/assets/NemoLogo.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/incubator-nemo/d384c79cb2e58481b0338cfc436c259b57dc88f9/webui/assets/NemoLogo.png -------------------------------------------------------------------------------- /webui/assets/constants.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | // valid state string 21 | export const STATE = { 22 | READY: 'READY', 23 | EXECUTING: 'EXECUTING', 24 | INCOMPLETE: 'INCOMPLETE', 25 | COMPLETE: 'COMPLETE', 26 | FAILED: 'FAILED', 27 | ON_HOLD: 'ON_HOLD', 28 | SHOULD_RETRY: 'SHOULD_RETRY', 29 | }; 30 | 31 | export const JOB_STATUS = { 32 | NOT_CONNECTED: 'NOT CONNECTED', 33 | CONNECTING: 'CONNECTING', 34 | RUNNING: 'RUNNING', 35 | COMPLETE: 'COMPLETE', 36 | FAILED: 'FAILED', 37 | }; 38 | -------------------------------------------------------------------------------- /webui/components/environment/EnvironmentView.vue: -------------------------------------------------------------------------------- 1 | 19 | 43 | 44 | 59 | -------------------------------------------------------------------------------- /webui/components/executors/ExecutorsView.vue: -------------------------------------------------------------------------------- 1 | 19 | 43 | 44 | 58 | -------------------------------------------------------------------------------- /webui/package.json: -------------------------------------------------------------------------------- 1 | { 2 | "name": "nemo-visualizer", 3 | "version": "1.0.0", 4 | "description": "Nemo Web Visualizer", 5 | "author": "Won Wook SONG , Jae Hyeon Park ", 6 | "private": true, 7 | "config": { 8 | "nuxt": { 9 | "host": "0.0.0.0", 10 | "port": "3333" 11 | } 12 | }, 13 | "scripts": { 14 | "dev": "nuxt", 15 | "build": "nuxt build", 16 | "start": "nuxt start", 17 | "generate": "nuxt generate", 18 | "lint": "eslint --ext .js,.vue --ignore-path .gitignore .", 19 | "precommit": "npm run lint", 20 | "test": "ava --serial --color --verbose" 21 | }, 22 | "ava": { 23 | "require": [ 24 | "babel-register", 25 | "./test/setup.js" 26 | ], 27 | "files": [ 28 | "./test/**/*.test.js" 29 | ] 30 | }, 31 | "babel": { 32 | "presets": [ 33 | "env" 34 | ] 35 | }, 36 | "dependencies": { 37 | "@dagrejs/graphlib": "^2.1.4", 38 | "dagre": "^0.8.2", 39 | "element-ui": "^2.4.9", 40 | "fabric": "^5.2.4", 41 | "npm": "^6.4.1", 42 | "nuxt": "^2.15.8", 43 | "uuid": "^3.3.2", 44 | "vue-affix": "^0.2.4", 45 | "vue2vis": "0.0.15" 46 | }, 47 | "devDependencies": { 48 | "@vue/test-utils": "^1.0.0-beta.25", 49 | "ava": "^0.25.0", 50 | "babel-eslint": "^8.2.6", 51 | "babel-polyfill": "^6.26.0", 52 | "eslint": "^4.15.0", 53 | "eslint-friendly-formatter": "^3.0.0", 54 | "eslint-loader": "^1.7.1", 55 | "eslint-plugin-vue": "^4.7.1", 56 | "require-extension-hooks": "^0.3.3", 57 | "require-extension-hooks-babel": "^0.1.1", 58 | "require-extension-hooks-vue": "^1.1.0" 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /webui/pages/index.vue: -------------------------------------------------------------------------------- 1 | 19 | 22 | 23 | 31 | 32 | 34 | -------------------------------------------------------------------------------- /webui/plugins/element-ui.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | import Vue from 'vue'; 20 | import ElementUI from 'element-ui'; 21 | import locale from 'element-ui/lib/locale/lang/en'; 22 | 23 | Vue.use(ElementUI, { locale }); 24 | -------------------------------------------------------------------------------- /webui/plugins/event-bus.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | import Vue from 'vue'; 20 | 21 | const eventBus = {}; 22 | 23 | eventBus.install = ( Vue ) => { 24 | Vue.prototype.$eventBus = new Vue(); 25 | }; 26 | 27 | Vue.use(eventBus); 28 | -------------------------------------------------------------------------------- /webui/plugins/vue-affix.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | import Vue from 'vue'; 20 | import VueAffix from 'vue-affix'; 21 | 22 | Vue.use(VueAffix); 23 | -------------------------------------------------------------------------------- /webui/plugins/vue2vis.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | import Vue from 'vue'; 20 | import { Timeline } from 'vue2vis'; 21 | 22 | Vue.component('timeline', Timeline); 23 | -------------------------------------------------------------------------------- /webui/static/favicon.ico: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/incubator-nemo/d384c79cb2e58481b0338cfc436c259b57dc88f9/webui/static/favicon.ico -------------------------------------------------------------------------------- /webui/static/nemo.ico: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/incubator-nemo/d384c79cb2e58481b0338cfc436c259b57dc88f9/webui/static/nemo.ico -------------------------------------------------------------------------------- /webui/test/setup.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | const hooks = require('require-extension-hooks'); 20 | 21 | require('babel-polyfill'); 22 | hooks('vue').plugin('vue').push(); 23 | hooks(['vue', 'js']).plugin('babel').push(); 24 | --------------------------------------------------------------------------------