You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nemo.apache.org by sa...@apache.org on 2018/08/06 03:01:14 UTC
[incubator-nemo] 01/01: Revert "[NEMO-167] Rename Passes and
Execution Properties (#83)"
This is an automated email from the ASF dual-hosted git repository.
sanha pushed a commit to branch revert-83-167-rename-passes
in repository https://gitbox.apache.org/repos/asf/incubator-nemo.git
commit 4d30cc08b06cd91170fc1ed033054adcaf9df3c6
Author: Sanha Lee <sa...@gmail.com>
AuthorDate: Mon Aug 6 12:01:11 2018 +0900
Revert "[NEMO-167] Rename Passes and Execution Properties (#83)"
This reverts commit e93facc3ecd8a3e85d844d03aa6f11299602c97f.
---
README.md | 6 +-
bin/json2dot.py | 2 +-
.../java/edu/snu/nemo/common/dag/DAGBuilder.java | 6 +-
.../java/edu/snu/nemo/common/ir/edge/IREdge.java | 6 +-
....java => DataCommunicationPatternProperty.java} | 10 +--
...lowProperty.java => DataFlowModelProperty.java} | 8 +-
...operty.java => InterTaskDataStoreProperty.java} | 8 +-
...Property.java => UsedDataHandlingProperty.java} | 13 +--
.../ir/executionproperty/ExecutionPropertyMap.java | 24 +++---
.../edu/snu/nemo/common/ir/vertex/LoopVertex.java | 10 +--
...roperty.java => ExecutorPlacementProperty.java} | 8 +-
...ty.java => ExecutorSlotComplianceProperty.java} | 12 +--
...rceSiteProperty.java => NodeNamesProperty.java} | 11 ++-
...y.java => SkewnessAwareSchedulingProperty.java} | 12 +--
... => SourceLocationAwareSchedulingProperty.java} | 12 +--
.../edu/snu/nemo/common/ir/LoopVertexTest.java | 20 ++---
.../ExecutionPropertyMapTest.java | 26 +++---
.../compiler/backend/nemo/NemoBackendTest.java | 14 ++--
.../frontend/beam/NemoPipelineVisitor.java | 16 ++--
.../frontend/spark/core/SparkFrontendUtils.java | 10 +--
.../MapReduceDisaggregationOptimization.java | 6 +-
...orePass.java => DataSkewEdgeDataStorePass.java} | 12 +--
....java => DataSkewEdgeMetricCollectionPass.java} | 12 +--
...rPass.java => DataSkewEdgePartitionerPass.java} | 6 +-
...SkewedDataPass.java => DataSkewVertexPass.java} | 10 +--
...s.java => DefaultEdgeUsedDataHandlingPass.java} | 26 +++---
...ass.java => DefaultInterTaskDataStorePass.java} | 12 +--
.../annotating/DefaultParallelismPass.java | 16 ++--
.../annotating/DefaultScheduleGroupPass.java | 24 +++---
.../DisaggregationEdgeDataStorePass.java | 6 +-
...otPass.java => ExecutorSlotCompliancePass.java} | 16 ++--
...eSitePass.java => NodeNamesAssignmentPass.java} | 34 ++++----
...lowPass.java => PadoEdgeDataFlowModelPass.java} | 18 ++--
...taStorePass.java => PadoEdgeDataStorePass.java} | 40 ++++-----
...s.java => PadoVertexExecutorPlacementPass.java} | 30 +++----
...ass.java => SailfishEdgeDataFlowModelPass.java} | 20 ++---
...orePass.java => SailfishEdgeDataStorePass.java} | 26 +++---
...coderPass.java => SailfishEdgeDecoderPass.java} | 14 ++--
...coderPass.java => SailfishEdgeEncoderPass.java} | 14 ++--
....java => SailfishEdgeUsedDataHandlingPass.java} | 20 ++---
... SailfishVertexExecutorSlotCompliancePass.java} | 28 +++----
.../annotating/ShuffleEdgePushPass.java | 12 +--
...java => SourceLocationAwareSchedulingPass.java} | 16 ++--
...mpositePass.java => DataSkewCompositePass.java} | 18 ++--
...ceCompositePass.java => PadoCompositePass.java} | 19 ++---
...positePass.java => PrimitiveCompositePass.java} | 14 ++--
...ShuffleCompositePass.java => SailfishPass.java} | 23 +++--
.../CommonSubexpressionEliminationPass.java | 6 +-
...shapingPass.java => DataSkewReshapingPass.java} | 22 ++---
.../compiletime/reshaping/LoopExtractionPass.java | 18 ++--
.../compiletime/reshaping/LoopOptimizations.java | 12 +--
...ngPass.java => SailfishRelayReshapingPass.java} | 20 ++---
.../compiler/optimizer/policy/DataSkewPolicy.java | 12 +--
.../compiler/optimizer/policy/DefaultPolicy.java | 4 +-
.../policy/DefaultPolicyWithSeparatePass.java | 4 +-
.../optimizer/policy/DisaggregationPolicy.java | 4 +-
...ransientResourcePolicy.java => PadoPolicy.java} | 15 ++--
.../compiler/optimizer/policy/PolicyBuilder.java | 16 ++--
...LargeShufflePolicy.java => SailfishPolicy.java} | 14 ++--
.../optimizer/policy/PolicyBuilderTest.java | 12 +--
.../compiler/backend/nemo/DAGConverterTest.java | 70 ++++++++--------
.../annotating/DefaultScheduleGroupPassTest.java | 44 +++++-----
...assTest.java => DataSkewCompositePassTest.java} | 24 +++---
.../composite/DisaggregationPassTest.java | 10 +--
.../composite/PadoCompositePassTest.java | 91 ++++++++++++++++++++
...ompositePassTest.java => SailfishPassTest.java} | 46 +++++-----
.../TransientResourceCompositePassTest.java | 91 --------------------
.../CommonSubexpressionEliminationPassTest.java | 18 ++--
.../compiletime/reshaping/LoopFusionPassTest.java | 6 +-
.../reshaping/LoopInvariantCodeMotionPassTest.java | 4 +-
.../beam/AlternatingLeastSquareITCase.java | 9 +-
.../snu/nemo/examples/beam/BroadcastITCase.java | 8 +-
.../examples/beam/NetworkTraceAnalysisITCase.java | 16 ++--
.../snu/nemo/examples/beam/WordCountITCase.java | 18 ++--
...ismFive.java => PadoPolicyParallelismFive.java} | 11 ++-
...elismTen.java => PadoPolicyParallelismTen.java} | 11 ++-
...ive.java => SailfishPolicyParallelismFive.java} | 11 ++-
.../runtime/common/plan/PhysicalPlanGenerator.java | 4 +-
.../snu/nemo/runtime/common/plan/StageEdge.java | 28 +++----
.../nemo/runtime/common/plan/StagePartitioner.java | 6 +-
.../common/plan/PhysicalPlanGeneratorTest.java | 14 ++--
.../runtime/common/plan/StagePartitionerTest.java | 29 ++++---
.../main/java/edu/snu/nemo/driver/NemoDriver.java | 4 +-
.../runtime/executor/data/BlockManagerWorker.java | 62 +++++++-------
.../runtime/executor/datatransfer/InputReader.java | 30 +++----
.../executor/datatransfer/OutputWriter.java | 16 ++--
.../executor/datatransfer/DataTransferTest.java | 97 +++++++++++-----------
.../runtime/executor/task/TaskExecutorTest.java | 6 +-
.../edu/snu/nemo/runtime/master/RuntimeMaster.java | 2 +-
.../master/resource/ExecutorRepresenter.java | 4 +-
.../ContainerTypeAwareSchedulingConstraint.java | 10 +--
.../scheduler/FreeSlotSchedulingConstraint.java | 6 +-
.../scheduler/NodeShareSchedulingConstraint.java | 12 +--
.../SkewnessAwareSchedulingConstraint.java | 4 +-
.../SourceLocationAwareSchedulingConstraint.java | 4 +-
.../nemo/runtime/master/ContainerManagerTest.java | 10 ++-
.../scheduler/BatchSingleJobSchedulerTest.java | 6 +-
...ContainerTypeAwareSchedulingConstraintTest.java | 16 ++--
.../FreeSlotSchedulingConstraintTest.java | 6 +-
.../SchedulingConstraintnRegistryTest.java | 12 +--
...ourceLocationAwareSchedulingConstraintTest.java | 4 +-
.../runtime/master/scheduler/TaskRetryTest.java | 4 +-
.../runtime/common/plan/TestPlanGenerator.java | 36 ++++----
103 files changed, 909 insertions(+), 896 deletions(-)
diff --git a/README.md b/README.md
index 3817f9f..b0ccb92 100644
--- a/README.md
+++ b/README.md
@@ -85,10 +85,10 @@ Please refer to the [Contribution guideline](.github/CONTRIBUTING.md) to contrib
## YARN cluster example
./bin/run_beam.sh \
-deploy_mode yarn \
- -job_id mr_transient \
+ -job_id mr_pado \
-executor_json `pwd`/examples/resources/beam_sample_executor_resources.json \
-user_main edu.snu.nemo.examples.beam.WordCount \
- -optimization_policy edu.snu.nemo.compiler.optimizer.policy.TransientResourcePolicy \
+ -optimization_policy edu.snu.nemo.compiler.optimizer.policy.PadoPolicy \
-user_args "hdfs://v-m:9000/sample_input_wordcount hdfs://v-m:9000/sample_output_wordcount"
```
## Resource Configuration
@@ -135,7 +135,7 @@ Nemo Compiler and Engine can store JSON representation of intermediate DAGs.
-job_id als \
-executor_json `pwd`/examples/resources/beam_sample_executor_resources.json \
-user_main edu.snu.nemo.examples.beam.AlternatingLeastSquare \
- -optimization_policy edu.snu.nemo.compiler.optimizer.policy.TransientResourcePolicy \
+ -optimization_policy edu.snu.nemo.compiler.optimizer.policy.PadoPolicy \
-dag_dir "./dag/als" \
-user_args "`pwd`/examples/resources/sample_input_als 10 3"
```
diff --git a/bin/json2dot.py b/bin/json2dot.py
index 582f30a..a0262ef 100755
--- a/bin/json2dot.py
+++ b/bin/json2dot.py
@@ -133,7 +133,7 @@ class NormalVertex:
def dot(self):
color = 'black'
try:
- placement = self.properties['executionProperties']['edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty']
+ placement = self.properties['executionProperties']['edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty']
if (placement == 'Transient'):
color = 'orange'
if (placement == 'Reserved'):
diff --git a/common/src/main/java/edu/snu/nemo/common/dag/DAGBuilder.java b/common/src/main/java/edu/snu/nemo/common/dag/DAGBuilder.java
index ffdd468..35c4330 100644
--- a/common/src/main/java/edu/snu/nemo/common/dag/DAGBuilder.java
+++ b/common/src/main/java/edu/snu/nemo/common/dag/DAGBuilder.java
@@ -16,12 +16,12 @@
package edu.snu.nemo.common.dag;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.MetricCollectionProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
import edu.snu.nemo.common.ir.vertex.SourceVertex;
import edu.snu.nemo.common.ir.vertex.LoopVertex;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
import edu.snu.nemo.common.exception.IllegalVertexOperationException;
import java.io.Serializable;
@@ -243,7 +243,7 @@ public final class DAGBuilder<V extends Vertex, E extends Edge<V>> implements Se
// SideInput is not compatible with Push
vertices.forEach(v -> incomingEdges.get(v).stream().filter(e -> e instanceof IREdge).map(e -> (IREdge) e)
.filter(e -> Boolean.TRUE.equals(e.isSideInput()))
- .filter(e -> DataFlowProperty.Value.Push.equals(e.getPropertyValue(DataFlowProperty.class).get()))
+ .filter(e -> DataFlowModelProperty.Value.Push.equals(e.getPropertyValue(DataFlowModelProperty.class).get()))
.forEach(e -> {
throw new RuntimeException("DAG execution property check: "
+ "SideInput edge is not compatible with push" + e.getId());
@@ -252,7 +252,7 @@ public final class DAGBuilder<V extends Vertex, E extends Edge<V>> implements Se
vertices.forEach(v -> incomingEdges.get(v).stream().filter(e -> e instanceof IREdge).map(e -> (IREdge) e)
.filter(e -> Optional.of(MetricCollectionProperty.Value.DataSkewRuntimePass)
.equals(e.getPropertyValue(MetricCollectionProperty.class)))
- .filter(e -> DataFlowProperty.Value.Push.equals(e.getPropertyValue(DataFlowProperty.class).get()))
+ .filter(e -> DataFlowModelProperty.Value.Push.equals(e.getPropertyValue(DataFlowModelProperty.class).get()))
.forEach(e -> {
throw new RuntimeException("DAG execution property check: "
+ "DataSizeMetricCollection edge is not compatible with push" + e.getId());
diff --git a/common/src/main/java/edu/snu/nemo/common/ir/edge/IREdge.java b/common/src/main/java/edu/snu/nemo/common/ir/edge/IREdge.java
index 080fc4b..b801841 100644
--- a/common/src/main/java/edu/snu/nemo/common/ir/edge/IREdge.java
+++ b/common/src/main/java/edu/snu/nemo/common/ir/edge/IREdge.java
@@ -17,7 +17,7 @@ package edu.snu.nemo.common.ir.edge;
import edu.snu.nemo.common.dag.Edge;
import edu.snu.nemo.common.ir.IdManager;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.executionproperty.EdgeExecutionProperty;
import edu.snu.nemo.common.ir.executionproperty.ExecutionPropertyMap;
import edu.snu.nemo.common.ir.vertex.IRVertex;
@@ -43,7 +43,7 @@ public final class IREdge extends Edge<IRVertex> {
* @param src source vertex.
* @param dst destination vertex.
*/
- public IREdge(final CommunicationPatternProperty.Value commPattern,
+ public IREdge(final DataCommunicationPatternProperty.Value commPattern,
final IRVertex src,
final IRVertex dst) {
this(commPattern, src, dst, false);
@@ -57,7 +57,7 @@ public final class IREdge extends Edge<IRVertex> {
* @param dst destination vertex.
* @param isSideInput flag for whether or not the edge is a sideInput.
*/
- public IREdge(final CommunicationPatternProperty.Value commPattern,
+ public IREdge(final DataCommunicationPatternProperty.Value commPattern,
final IRVertex src,
final IRVertex dst,
final Boolean isSideInput) {
diff --git a/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/CommunicationPatternProperty.java b/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataCommunicationPatternProperty.java
similarity index 80%
rename from common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/CommunicationPatternProperty.java
rename to common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataCommunicationPatternProperty.java
index ae149fe..cab9b77 100644
--- a/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/CommunicationPatternProperty.java
+++ b/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataCommunicationPatternProperty.java
@@ -21,13 +21,13 @@ import edu.snu.nemo.common.ir.executionproperty.EdgeExecutionProperty;
/**
* DataCommunicationPattern ExecutionProperty.
*/
-public final class CommunicationPatternProperty
- extends EdgeExecutionProperty<CommunicationPatternProperty.Value> {
+public final class DataCommunicationPatternProperty
+ extends EdgeExecutionProperty<DataCommunicationPatternProperty.Value> {
/**
* Constructor.
* @param value value of the execution property.
*/
- private CommunicationPatternProperty(final Value value) {
+ private DataCommunicationPatternProperty(final Value value) {
super(value);
}
@@ -36,8 +36,8 @@ public final class CommunicationPatternProperty
* @param value value of the new execution property.
* @return the newly created execution property.
*/
- public static CommunicationPatternProperty of(final Value value) {
- return new CommunicationPatternProperty(value);
+ public static DataCommunicationPatternProperty of(final Value value) {
+ return new DataCommunicationPatternProperty(value);
}
/**
diff --git a/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataFlowProperty.java b/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataFlowModelProperty.java
similarity index 81%
rename from common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataFlowProperty.java
rename to common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataFlowModelProperty.java
index fb9312d..1c1a8bb 100644
--- a/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataFlowProperty.java
+++ b/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataFlowModelProperty.java
@@ -20,12 +20,12 @@ import edu.snu.nemo.common.ir.executionproperty.EdgeExecutionProperty;
/**
* DataFlowModel ExecutionProperty.
*/
-public final class DataFlowProperty extends EdgeExecutionProperty<DataFlowProperty.Value> {
+public final class DataFlowModelProperty extends EdgeExecutionProperty<DataFlowModelProperty.Value> {
/**
* Constructor.
* @param value value of the execution property.
*/
- private DataFlowProperty(final Value value) {
+ private DataFlowModelProperty(final Value value) {
super(value);
}
@@ -34,8 +34,8 @@ public final class DataFlowProperty extends EdgeExecutionProperty<DataFlowProper
* @param value value of the new execution property.
* @return the newly created execution property.
*/
- public static DataFlowProperty of(final Value value) {
- return new DataFlowProperty(value);
+ public static DataFlowModelProperty of(final Value value) {
+ return new DataFlowModelProperty(value);
}
/**
diff --git a/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataStoreProperty.java b/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/InterTaskDataStoreProperty.java
similarity index 81%
rename from common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataStoreProperty.java
rename to common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/InterTaskDataStoreProperty.java
index 6e87844..af6c85c 100644
--- a/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataStoreProperty.java
+++ b/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/InterTaskDataStoreProperty.java
@@ -20,12 +20,12 @@ import edu.snu.nemo.common.ir.executionproperty.EdgeExecutionProperty;
/**
* DataStore ExecutionProperty.
*/
-public final class DataStoreProperty extends EdgeExecutionProperty<DataStoreProperty.Value> {
+public final class InterTaskDataStoreProperty extends EdgeExecutionProperty<InterTaskDataStoreProperty.Value> {
/**
* Constructor.
* @param value value of the execution property.
*/
- private DataStoreProperty(final Value value) {
+ private InterTaskDataStoreProperty(final Value value) {
super(value);
}
@@ -34,8 +34,8 @@ public final class DataStoreProperty extends EdgeExecutionProperty<DataStoreProp
* @param value value of the new execution property.
* @return the newly created execution property.
*/
- public static DataStoreProperty of(final Value value) {
- return new DataStoreProperty(value);
+ public static InterTaskDataStoreProperty of(final Value value) {
+ return new InterTaskDataStoreProperty(value);
}
/**
diff --git a/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataPersistenceProperty.java b/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/UsedDataHandlingProperty.java
similarity index 71%
rename from common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataPersistenceProperty.java
rename to common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/UsedDataHandlingProperty.java
index a1d96b2..33f6f74 100644
--- a/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/DataPersistenceProperty.java
+++ b/common/src/main/java/edu/snu/nemo/common/ir/edge/executionproperty/UsedDataHandlingProperty.java
@@ -18,14 +18,15 @@ package edu.snu.nemo.common.ir.edge.executionproperty;
import edu.snu.nemo.common.ir.executionproperty.EdgeExecutionProperty;
/**
- * Data persistence ExecutionProperty.
+ * UsedDataHandling ExecutionProperty.
+ * This property represents the used data handling strategy.
*/
-public final class DataPersistenceProperty extends EdgeExecutionProperty<DataPersistenceProperty.Value> {
+public final class UsedDataHandlingProperty extends EdgeExecutionProperty<UsedDataHandlingProperty.Value> {
/**
* Constructor.
* @param value value of the execution property.
*/
- private DataPersistenceProperty(final Value value) {
+ private UsedDataHandlingProperty(final Value value) {
super(value);
}
@@ -34,12 +35,12 @@ public final class DataPersistenceProperty extends EdgeExecutionProperty<DataPer
* @param value value of the new execution property.
* @return the newly created execution property.
*/
- public static DataPersistenceProperty of(final Value value) {
- return new DataPersistenceProperty(value);
+ public static UsedDataHandlingProperty of(final Value value) {
+ return new UsedDataHandlingProperty(value);
}
/**
- * Possible options for the data persistence strategy.
+ * Possible values of UsedDataHandling ExecutionProperty.
*/
public enum Value {
Discard,
diff --git a/common/src/main/java/edu/snu/nemo/common/ir/executionproperty/ExecutionPropertyMap.java b/common/src/main/java/edu/snu/nemo/common/ir/executionproperty/ExecutionPropertyMap.java
index a2ef8d9..fdbd5dc 100644
--- a/common/src/main/java/edu/snu/nemo/common/ir/executionproperty/ExecutionPropertyMap.java
+++ b/common/src/main/java/edu/snu/nemo/common/ir/executionproperty/ExecutionPropertyMap.java
@@ -16,12 +16,12 @@
package edu.snu.nemo.common.ir.executionproperty;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.PartitionerProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
import edu.snu.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
import com.google.common.annotations.VisibleForTesting;
@@ -60,26 +60,26 @@ public final class ExecutionPropertyMap<T extends ExecutionProperty> implements
*/
public static ExecutionPropertyMap<EdgeExecutionProperty> of(
final IREdge irEdge,
- final CommunicationPatternProperty.Value commPattern) {
+ final DataCommunicationPatternProperty.Value commPattern) {
final ExecutionPropertyMap<EdgeExecutionProperty> map = new ExecutionPropertyMap<>(irEdge.getId());
- map.put(CommunicationPatternProperty.of(commPattern));
- map.put(DataFlowProperty.of(DataFlowProperty.Value.Pull));
+ map.put(DataCommunicationPatternProperty.of(commPattern));
+ map.put(DataFlowModelProperty.of(DataFlowModelProperty.Value.Pull));
switch (commPattern) {
case Shuffle:
map.put(PartitionerProperty.of(PartitionerProperty.Value.HashPartitioner));
- map.put(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore));
+ map.put(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.LocalFileStore));
break;
case BroadCast:
map.put(PartitionerProperty.of(PartitionerProperty.Value.IntactPartitioner));
- map.put(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore));
+ map.put(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.LocalFileStore));
break;
case OneToOne:
map.put(PartitionerProperty.of(PartitionerProperty.Value.IntactPartitioner));
- map.put(DataStoreProperty.of(DataStoreProperty.Value.MemoryStore));
+ map.put(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.MemoryStore));
break;
default:
map.put(PartitionerProperty.of(PartitionerProperty.Value.HashPartitioner));
- map.put(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore));
+ map.put(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.LocalFileStore));
}
return map;
}
@@ -92,7 +92,7 @@ public final class ExecutionPropertyMap<T extends ExecutionProperty> implements
public static ExecutionPropertyMap<VertexExecutionProperty> of(final IRVertex irVertex) {
final ExecutionPropertyMap<VertexExecutionProperty> map = new ExecutionPropertyMap<>(irVertex.getId());
map.put(ParallelismProperty.of(1));
- map.put(ResourcePriorityProperty.of(ResourcePriorityProperty.NONE));
+ map.put(ExecutorPlacementProperty.of(ExecutorPlacementProperty.NONE));
return map;
}
diff --git a/common/src/main/java/edu/snu/nemo/common/ir/vertex/LoopVertex.java b/common/src/main/java/edu/snu/nemo/common/ir/vertex/LoopVertex.java
index 96c7fca..a761c19 100644
--- a/common/src/main/java/edu/snu/nemo/common/ir/vertex/LoopVertex.java
+++ b/common/src/main/java/edu/snu/nemo/common/ir/vertex/LoopVertex.java
@@ -18,7 +18,7 @@ package edu.snu.nemo.common.ir.vertex;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DuplicateEdgeGroupProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DuplicateEdgeGroupPropertyValue;
@@ -217,7 +217,7 @@ public final class LoopVertex extends IRVertex {
dagBuilder.addVertex(newIrVertex, dagToAdd);
dagToAdd.getIncomingEdgesOf(irVertex).forEach(edge -> {
final IRVertex newSrc = originalToNewIRVertex.get(edge.getSrc());
- final IREdge newIrEdge = new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge newIrEdge = new IREdge(edge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
newSrc, newIrVertex, edge.isSideInput());
edge.copyExecutionPropertiesTo(newIrEdge);
dagBuilder.connectVertices(newIrEdge);
@@ -226,7 +226,7 @@ public final class LoopVertex extends IRVertex {
// process DAG incoming edges.
getDagIncomingEdges().forEach((dstVertex, irEdges) -> irEdges.forEach(edge -> {
- final IREdge newIrEdge = new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge newIrEdge = new IREdge(edge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
edge.getSrc(), originalToNewIRVertex.get(dstVertex), edge.isSideInput());
edge.copyExecutionPropertiesTo(newIrEdge);
dagBuilder.connectVertices(newIrEdge);
@@ -235,7 +235,7 @@ public final class LoopVertex extends IRVertex {
if (loopTerminationConditionMet()) {
// if termination condition met, we process the DAG outgoing edge.
getDagOutgoingEdges().forEach((srcVertex, irEdges) -> irEdges.forEach(edge -> {
- final IREdge newIrEdge = new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge newIrEdge = new IREdge(edge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
originalToNewIRVertex.get(srcVertex), edge.getDst(), edge.isSideInput());
edge.copyExecutionPropertiesTo(newIrEdge);
dagBuilder.addVertex(edge.getDst()).connectVertices(newIrEdge);
@@ -246,7 +246,7 @@ public final class LoopVertex extends IRVertex {
this.getDagIncomingEdges().clear();
this.nonIterativeIncomingEdges.forEach((dstVertex, irEdges) -> irEdges.forEach(this::addDagIncomingEdge));
this.iterativeIncomingEdges.forEach((dstVertex, irEdges) -> irEdges.forEach(edge -> {
- final IREdge newIrEdge = new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge newIrEdge = new IREdge(edge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
originalToNewIRVertex.get(edge.getSrc()), dstVertex, edge.isSideInput());
edge.copyExecutionPropertiesTo(newIrEdge);
this.addDagIncomingEdge(newIrEdge);
diff --git a/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourcePriorityProperty.java b/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ExecutorPlacementProperty.java
similarity index 83%
rename from common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourcePriorityProperty.java
rename to common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ExecutorPlacementProperty.java
index 26bfd34..af22ed0 100644
--- a/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourcePriorityProperty.java
+++ b/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ExecutorPlacementProperty.java
@@ -20,12 +20,12 @@ import edu.snu.nemo.common.ir.executionproperty.VertexExecutionProperty;
/**
* ExecutionPlacement ExecutionProperty.
*/
-public final class ResourcePriorityProperty extends VertexExecutionProperty<String> {
+public final class ExecutorPlacementProperty extends VertexExecutionProperty<String> {
/**
* Constructor.
* @param value value of the execution property.
*/
- private ResourcePriorityProperty(final String value) {
+ private ExecutorPlacementProperty(final String value) {
super(value);
}
@@ -34,8 +34,8 @@ public final class ResourcePriorityProperty extends VertexExecutionProperty<Stri
* @param value value of the new execution property.
* @return the newly created execution property.
*/
- public static ResourcePriorityProperty of(final String value) {
- return new ResourcePriorityProperty(value);
+ public static ExecutorPlacementProperty of(final String value) {
+ return new ExecutorPlacementProperty(value);
}
// List of default pre-configured values.
diff --git a/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourceSlotProperty.java b/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ExecutorSlotComplianceProperty.java
similarity index 71%
rename from common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourceSlotProperty.java
rename to common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ExecutorSlotComplianceProperty.java
index f82d958..357be21 100644
--- a/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourceSlotProperty.java
+++ b/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ExecutorSlotComplianceProperty.java
@@ -20,17 +20,17 @@ import edu.snu.nemo.common.ir.executionproperty.VertexExecutionProperty;
/**
* This property decides whether or not to comply to slot restrictions when scheduling this vertex.
*/
-public final class ResourceSlotProperty extends VertexExecutionProperty<Boolean> {
- private static final ResourceSlotProperty COMPLIANCE_TRUE = new ResourceSlotProperty(true);
- private static final ResourceSlotProperty COMPLIANCE_FALSE
- = new ResourceSlotProperty(false);
+public final class ExecutorSlotComplianceProperty extends VertexExecutionProperty<Boolean> {
+ private static final ExecutorSlotComplianceProperty COMPLIANCE_TRUE = new ExecutorSlotComplianceProperty(true);
+ private static final ExecutorSlotComplianceProperty COMPLIANCE_FALSE
+ = new ExecutorSlotComplianceProperty(false);
/**
* Default constructor.
*
* @param value value of the ExecutionProperty
*/
- private ResourceSlotProperty(final boolean value) {
+ private ExecutorSlotComplianceProperty(final boolean value) {
super(value);
}
@@ -40,7 +40,7 @@ public final class ResourceSlotProperty extends VertexExecutionProperty<Boolean>
* @param value value of the new execution property
* @return the execution property
*/
- public static ResourceSlotProperty of(final boolean value) {
+ public static ExecutorSlotComplianceProperty of(final boolean value) {
return value ? COMPLIANCE_TRUE : COMPLIANCE_FALSE;
}
}
diff --git a/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourceSiteProperty.java b/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/NodeNamesProperty.java
similarity index 73%
rename from common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourceSiteProperty.java
rename to common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/NodeNamesProperty.java
index c42f741..82cddd9 100644
--- a/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourceSiteProperty.java
+++ b/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/NodeNamesProperty.java
@@ -21,23 +21,22 @@ import java.util.HashMap;
/**
* Map between node name and the number of parallelism which will run on the node.
- * TODO #169: Use sites (not node names) in ResourceSiteProperty
*/
-public final class ResourceSiteProperty extends VertexExecutionProperty<HashMap<String, Integer>> {
+public final class NodeNamesProperty extends VertexExecutionProperty<HashMap<String, Integer>> {
/**
* Default constructor.
* @param value the map from location to the number of Task that must be executed on the node
*/
- public ResourceSiteProperty(final HashMap<String, Integer> value) {
+ public NodeNamesProperty(final HashMap<String, Integer> value) {
super(value);
}
/**
- * Static method for constructing {@link ResourceSiteProperty}.
+ * Static method for constructing {@link NodeNamesProperty}.
* @param value the map from location to the number of Task that must be executed on the node
* @return the execution property
*/
- public static ResourceSiteProperty of(final HashMap<String, Integer> value) {
- return new ResourceSiteProperty(value);
+ public static NodeNamesProperty of(final HashMap<String, Integer> value) {
+ return new NodeNamesProperty(value);
}
}
diff --git a/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourceSkewedDataProperty.java b/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/SkewnessAwareSchedulingProperty.java
similarity index 71%
rename from common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourceSkewedDataProperty.java
rename to common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/SkewnessAwareSchedulingProperty.java
index dadc6b3..9ec3364 100644
--- a/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourceSkewedDataProperty.java
+++ b/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/SkewnessAwareSchedulingProperty.java
@@ -20,16 +20,18 @@ import edu.snu.nemo.common.ir.executionproperty.VertexExecutionProperty;
/**
* This property decides whether or not to handle skew when scheduling this vertex.
*/
-public final class ResourceSkewedDataProperty extends VertexExecutionProperty<Boolean> {
- private static final ResourceSkewedDataProperty HANDLE_SKEW = new ResourceSkewedDataProperty(true);
- private static final ResourceSkewedDataProperty DONT_HANDLE_SKEW = new ResourceSkewedDataProperty(false);
+public final class SkewnessAwareSchedulingProperty extends VertexExecutionProperty<Boolean> {
+ private static final SkewnessAwareSchedulingProperty HANDLE_SKEW
+ = new SkewnessAwareSchedulingProperty(true);
+ private static final SkewnessAwareSchedulingProperty DONT_HANDLE_SKEW
+ = new SkewnessAwareSchedulingProperty(false);
/**
* Default constructor.
*
* @param value value of the ExecutionProperty
*/
- private ResourceSkewedDataProperty(final boolean value) {
+ private SkewnessAwareSchedulingProperty(final boolean value) {
super(value);
}
@@ -39,7 +41,7 @@ public final class ResourceSkewedDataProperty extends VertexExecutionProperty<Bo
* @param value value of the new execution property
* @return the execution property
*/
- public static ResourceSkewedDataProperty of(final boolean value) {
+ public static SkewnessAwareSchedulingProperty of(final boolean value) {
return value ? HANDLE_SKEW : DONT_HANDLE_SKEW;
}
}
diff --git a/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourceLocalityProperty.java b/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/SourceLocationAwareSchedulingProperty.java
similarity index 69%
rename from common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourceLocalityProperty.java
rename to common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/SourceLocationAwareSchedulingProperty.java
index 869644e..cad987d 100644
--- a/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/ResourceLocalityProperty.java
+++ b/common/src/main/java/edu/snu/nemo/common/ir/vertex/executionproperty/SourceLocationAwareSchedulingProperty.java
@@ -20,16 +20,18 @@ import edu.snu.nemo.common.ir.executionproperty.VertexExecutionProperty;
/**
* This property decides whether or not to schedule this vertex only on executors where source data reside.
*/
-public final class ResourceLocalityProperty extends VertexExecutionProperty<Boolean> {
- private static final ResourceLocalityProperty SOURCE_TRUE = new ResourceLocalityProperty(true);
- private static final ResourceLocalityProperty SOURCE_FALSE = new ResourceLocalityProperty(false);
+public final class SourceLocationAwareSchedulingProperty extends VertexExecutionProperty<Boolean> {
+ private static final SourceLocationAwareSchedulingProperty SOURCE_TRUE
+ = new SourceLocationAwareSchedulingProperty(true);
+ private static final SourceLocationAwareSchedulingProperty SOURCE_FALSE
+ = new SourceLocationAwareSchedulingProperty(false);
/**
* Default constructor.
*
* @param value value of the ExecutionProperty
*/
- private ResourceLocalityProperty(final boolean value) {
+ private SourceLocationAwareSchedulingProperty(final boolean value) {
super(value);
}
@@ -39,7 +41,7 @@ public final class ResourceLocalityProperty extends VertexExecutionProperty<Bool
* @param value value of the new execution property
* @return the execution property
*/
- public static ResourceLocalityProperty of(final boolean value) {
+ public static SourceLocationAwareSchedulingProperty of(final boolean value) {
return value ? SOURCE_TRUE : SOURCE_FALSE;
}
}
diff --git a/common/src/test/java/edu/snu/nemo/common/ir/LoopVertexTest.java b/common/src/test/java/edu/snu/nemo/common/ir/LoopVertexTest.java
index d3293c6..558a9f7 100644
--- a/common/src/test/java/edu/snu/nemo/common/ir/LoopVertexTest.java
+++ b/common/src/test/java/edu/snu/nemo/common/ir/LoopVertexTest.java
@@ -17,7 +17,7 @@ package edu.snu.nemo.common.ir;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.LoopVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
@@ -53,17 +53,17 @@ public class LoopVertexTest {
final DAGBuilder<IRVertex, IREdge> builder = new DAGBuilder<>();
loopDAGBuilder.addVertex(map1).addVertex(groupByKey).addVertex(combine).addVertex(map2)
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.Shuffle, map1, groupByKey))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, groupByKey, combine))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, combine, map2));
- loopVertex.addDagIncomingEdge(new IREdge(CommunicationPatternProperty.Value.OneToOne, source, map1));
- loopVertex.addIterativeIncomingEdge(new IREdge(CommunicationPatternProperty.Value.OneToOne, map2, map1));
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.Shuffle, map1, groupByKey))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, groupByKey, combine))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, combine, map2));
+ loopVertex.addDagIncomingEdge(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, source, map1));
+ loopVertex.addIterativeIncomingEdge(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, map2, map1));
originalDAG = builder.addVertex(source).addVertex(map1).addVertex(groupByKey).addVertex(combine).addVertex(map2)
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, source, map1))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.Shuffle, map1, groupByKey))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, groupByKey, combine))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, combine, map2))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, source, map1))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.Shuffle, map1, groupByKey))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, groupByKey, combine))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, combine, map2))
.build();
}
diff --git a/common/src/test/java/edu/snu/nemo/common/ir/executionproperty/ExecutionPropertyMapTest.java b/common/src/test/java/edu/snu/nemo/common/ir/executionproperty/ExecutionPropertyMapTest.java
index d5aba03..ed27235 100644
--- a/common/src/test/java/edu/snu/nemo/common/ir/executionproperty/ExecutionPropertyMapTest.java
+++ b/common/src/test/java/edu/snu/nemo/common/ir/executionproperty/ExecutionPropertyMapTest.java
@@ -36,21 +36,21 @@ import static org.junit.Assert.assertTrue;
public class ExecutionPropertyMapTest {
private final IRVertex source = new EmptyComponents.EmptySourceVertex<>("Source");
private final IRVertex destination = new OperatorVertex(new EmptyComponents.EmptyTransform("MapElements"));
- private final CommunicationPatternProperty.Value comPattern = CommunicationPatternProperty.Value.OneToOne;
- private final IREdge edge = new IREdge(CommunicationPatternProperty.Value.OneToOne, source, destination);
+ private final DataCommunicationPatternProperty.Value comPattern = DataCommunicationPatternProperty.Value.OneToOne;
+ private final IREdge edge = new IREdge(DataCommunicationPatternProperty.Value.OneToOne, source, destination);
private ExecutionPropertyMap<EdgeExecutionProperty> edgeMap;
private ExecutionPropertyMap<VertexExecutionProperty> vertexMap;
@Before
public void setUp() {
- this.edgeMap = ExecutionPropertyMap.of(edge, CommunicationPatternProperty.Value.OneToOne);
+ this.edgeMap = ExecutionPropertyMap.of(edge, DataCommunicationPatternProperty.Value.OneToOne);
this.vertexMap = ExecutionPropertyMap.of(source);
}
@Test
public void testDefaultValues() {
- assertEquals(comPattern, edgeMap.get(CommunicationPatternProperty.class).get());
+ assertEquals(comPattern, edgeMap.get(DataCommunicationPatternProperty.class).get());
assertEquals(1, vertexMap.get(ParallelismProperty.class).get().longValue());
assertEquals(edge.getId(), edgeMap.getId());
assertEquals(source.getId(), vertexMap.getId());
@@ -58,17 +58,17 @@ public class ExecutionPropertyMapTest {
@Test
public void testPutGetAndRemove() {
- edgeMap.put(DataStoreProperty.of(DataStoreProperty.Value.MemoryStore));
- assertEquals(DataStoreProperty.Value.MemoryStore, edgeMap.get(DataStoreProperty.class).get());
- edgeMap.put(DataFlowProperty.of(DataFlowProperty.Value.Pull));
- assertEquals(DataFlowProperty.Value.Pull, edgeMap.get(DataFlowProperty.class).get());
+ edgeMap.put(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.MemoryStore));
+ assertEquals(InterTaskDataStoreProperty.Value.MemoryStore, edgeMap.get(InterTaskDataStoreProperty.class).get());
+ edgeMap.put(DataFlowModelProperty.of(DataFlowModelProperty.Value.Pull));
+ assertEquals(DataFlowModelProperty.Value.Pull, edgeMap.get(DataFlowModelProperty.class).get());
edgeMap.put(EncoderProperty.of(EncoderFactory.DUMMY_ENCODER_FACTORY));
assertEquals(EncoderFactory.DUMMY_ENCODER_FACTORY, edgeMap.get(EncoderProperty.class).get());
edgeMap.put(DecoderProperty.of(DecoderFactory.DUMMY_DECODER_FACTORY));
assertEquals(DecoderFactory.DUMMY_DECODER_FACTORY, edgeMap.get(DecoderProperty.class).get());
- edgeMap.remove(DataFlowProperty.class);
- assertFalse(edgeMap.get(DataFlowProperty.class).isPresent());
+ edgeMap.remove(DataFlowModelProperty.class);
+ assertFalse(edgeMap.get(DataFlowModelProperty.class).isPresent());
vertexMap.put(ParallelismProperty.of(100));
assertEquals(100, vertexMap.get(ParallelismProperty.class).get().longValue());
@@ -92,13 +92,13 @@ public class ExecutionPropertyMapTest {
map0.put(ParallelismProperty.of(2));
assertTrue(map0.equals(map1));
assertTrue(map1.equals(map0));
- map0.put(DataFlowProperty.of(DataFlowProperty.Value.Pull));
+ map0.put(DataFlowModelProperty.of(DataFlowModelProperty.Value.Pull));
assertFalse(map0.equals(map1));
assertFalse(map1.equals(map0));
- map1.put(DataFlowProperty.of(DataFlowProperty.Value.Push));
+ map1.put(DataFlowModelProperty.of(DataFlowModelProperty.Value.Push));
assertFalse(map0.equals(map1));
assertFalse(map1.equals(map0));
- map1.put(DataFlowProperty.of(DataFlowProperty.Value.Pull));
+ map1.put(DataFlowModelProperty.of(DataFlowModelProperty.Value.Pull));
assertTrue(map0.equals(map1));
assertTrue(map1.equals(map0));
}
diff --git a/compiler/backend/src/test/java/edu/snu/nemo/compiler/backend/nemo/NemoBackendTest.java b/compiler/backend/src/test/java/edu/snu/nemo/compiler/backend/nemo/NemoBackendTest.java
index f8435e4..f93a9b6 100644
--- a/compiler/backend/src/test/java/edu/snu/nemo/compiler/backend/nemo/NemoBackendTest.java
+++ b/compiler/backend/src/test/java/edu/snu/nemo/compiler/backend/nemo/NemoBackendTest.java
@@ -17,13 +17,13 @@ package edu.snu.nemo.compiler.backend.nemo;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.compiler.optimizer.CompiletimeOptimizer;
import edu.snu.nemo.common.test.EmptyComponents;
-import edu.snu.nemo.compiler.optimizer.policy.TransientResourcePolicy;
+import edu.snu.nemo.compiler.optimizer.policy.PadoPolicy;
import edu.snu.nemo.conf.JobConf;
import edu.snu.nemo.runtime.common.plan.PhysicalPlan;
import edu.snu.nemo.runtime.common.plan.PhysicalPlanGenerator;
@@ -53,13 +53,13 @@ public final class NemoBackendTest<I, O> {
@Before
public void setUp() throws Exception {
this.dag = builder.addVertex(source).addVertex(map1).addVertex(groupByKey).addVertex(combine).addVertex(map2)
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, source, map1))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.Shuffle, map1, groupByKey))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, groupByKey, combine))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, combine, map2))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, source, map1))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.Shuffle, map1, groupByKey))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, groupByKey, combine))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, combine, map2))
.build();
- this.dag = CompiletimeOptimizer.optimize(dag, new TransientResourcePolicy(), EMPTY_DAG_DIRECTORY);
+ this.dag = CompiletimeOptimizer.optimize(dag, new PadoPolicy(), EMPTY_DAG_DIRECTORY);
final Injector injector = Tang.Factory.getTang().newInjector();
injector.bindVolatileParameter(JobConf.DAGDirectory.class, "");
diff --git a/compiler/frontend/beam/src/main/java/edu/snu/nemo/compiler/frontend/beam/NemoPipelineVisitor.java b/compiler/frontend/beam/src/main/java/edu/snu/nemo/compiler/frontend/beam/NemoPipelineVisitor.java
index ecf6c79..39f14bb 100644
--- a/compiler/frontend/beam/src/main/java/edu/snu/nemo/compiler/frontend/beam/NemoPipelineVisitor.java
+++ b/compiler/frontend/beam/src/main/java/edu/snu/nemo/compiler/frontend/beam/NemoPipelineVisitor.java
@@ -16,7 +16,6 @@
package edu.snu.nemo.compiler.frontend.beam;
import edu.snu.nemo.common.Pair;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DecoderProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.EncoderProperty;
import edu.snu.nemo.common.ir.vertex.executionproperty.AdditionalTagOutputProperty;
@@ -25,6 +24,7 @@ import edu.snu.nemo.compiler.frontend.beam.coder.BeamDecoderFactory;
import edu.snu.nemo.compiler.frontend.beam.coder.BeamEncoderFactory;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.compiler.frontend.beam.source.BeamBoundedSourceVertex;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.LoopVertex;
@@ -280,8 +280,8 @@ public final class NemoPipelineVisitor extends Pipeline.PipelineVisitor.Defaults
* @param dst destination vertex.
* @return the appropriate edge type.
*/
- private static CommunicationPatternProperty.Value getEdgeCommunicationPattern(final IRVertex src,
- final IRVertex dst) {
+ private static DataCommunicationPatternProperty.Value getEdgeCommunicationPattern(final IRVertex src,
+ final IRVertex dst) {
final Class<?> constructUnionTableFn;
try {
constructUnionTableFn = Class.forName("org.apache.beam.sdk.transforms.join.CoGroupByKey$ConstructUnionTableFn");
@@ -294,17 +294,17 @@ public final class NemoPipelineVisitor extends Pipeline.PipelineVisitor.Defaults
final DoFn srcDoFn = srcTransform instanceof DoTransform ? ((DoTransform) srcTransform).getDoFn() : null;
if (srcDoFn != null && srcDoFn.getClass().equals(constructUnionTableFn)) {
- return CommunicationPatternProperty.Value.Shuffle;
+ return DataCommunicationPatternProperty.Value.Shuffle;
}
if (srcTransform instanceof FlattenTransform) {
- return CommunicationPatternProperty.Value.OneToOne;
+ return DataCommunicationPatternProperty.Value.OneToOne;
}
if (dstTransform instanceof GroupByKeyTransform) {
- return CommunicationPatternProperty.Value.Shuffle;
+ return DataCommunicationPatternProperty.Value.Shuffle;
}
if (dstTransform instanceof CreateViewTransform) {
- return CommunicationPatternProperty.Value.BroadCast;
+ return DataCommunicationPatternProperty.Value.BroadCast;
}
- return CommunicationPatternProperty.Value.OneToOne;
+ return DataCommunicationPatternProperty.Value.OneToOne;
}
}
diff --git a/compiler/frontend/spark/src/main/java/edu/snu/nemo/compiler/frontend/spark/core/SparkFrontendUtils.java b/compiler/frontend/spark/src/main/java/edu/snu/nemo/compiler/frontend/spark/core/SparkFrontendUtils.java
index 54de1a6..33fedd1 100644
--- a/compiler/frontend/spark/src/main/java/edu/snu/nemo/compiler/frontend/spark/core/SparkFrontendUtils.java
+++ b/compiler/frontend/spark/src/main/java/edu/snu/nemo/compiler/frontend/spark/core/SparkFrontendUtils.java
@@ -19,9 +19,9 @@ import edu.snu.nemo.client.JobLauncher;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DecoderProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.EncoderProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.KeyExtractorProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.LoopVertex;
@@ -112,14 +112,14 @@ public final class SparkFrontendUtils {
* @param dst destination vertex.
* @return the communication pattern.
*/
- public static CommunicationPatternProperty.Value getEdgeCommunicationPattern(final IRVertex src,
- final IRVertex dst) {
+ public static DataCommunicationPatternProperty.Value getEdgeCommunicationPattern(final IRVertex src,
+ final IRVertex dst) {
if (dst instanceof OperatorVertex
&& (((OperatorVertex) dst).getTransform() instanceof ReduceByKeyTransform
|| ((OperatorVertex) dst).getTransform() instanceof GroupByKeyTransform)) {
- return CommunicationPatternProperty.Value.Shuffle;
+ return DataCommunicationPatternProperty.Value.Shuffle;
} else {
- return CommunicationPatternProperty.Value.OneToOne;
+ return DataCommunicationPatternProperty.Value.OneToOne;
}
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/examples/MapReduceDisaggregationOptimization.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/examples/MapReduceDisaggregationOptimization.java
index 81de704..6b3bd40 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/examples/MapReduceDisaggregationOptimization.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/examples/MapReduceDisaggregationOptimization.java
@@ -15,7 +15,7 @@
*/
package edu.snu.nemo.compiler.optimizer.examples;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
@@ -58,10 +58,10 @@ public final class MapReduceDisaggregationOptimization {
builder.addVertex(map);
builder.addVertex(reduce);
- final IREdge edge1 = new IREdge(CommunicationPatternProperty.Value.OneToOne, source, map);
+ final IREdge edge1 = new IREdge(DataCommunicationPatternProperty.Value.OneToOne, source, map);
builder.connectVertices(edge1);
- final IREdge edge2 = new IREdge(CommunicationPatternProperty.Value.Shuffle, map, reduce);
+ final IREdge edge2 = new IREdge(DataCommunicationPatternProperty.Value.Shuffle, map, reduce);
builder.connectVertices(edge2);
final DAG<IRVertex, IREdge> dag = builder.build();
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SkewDataStorePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DataSkewEdgeDataStorePass.java
similarity index 81%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SkewDataStorePass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DataSkewEdgeDataStorePass.java
index 6405c45..857069b 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SkewDataStorePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DataSkewEdgeDataStorePass.java
@@ -17,7 +17,7 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.MetricCollectionBarrierVertex;
@@ -26,12 +26,12 @@ import edu.snu.nemo.common.ir.vertex.MetricCollectionBarrierVertex;
* It specifies the incoming one-to-one edges to MetricCollectionVertices to have either MemoryStore or LocalFileStore
* as its DataStore ExecutionProperty.
*/
-public final class SkewDataStorePass extends AnnotatingPass {
+public final class DataSkewEdgeDataStorePass extends AnnotatingPass {
/**
* Default constructor.
*/
- public SkewDataStorePass() {
- super(DataStoreProperty.class);
+ public DataSkewEdgeDataStorePass() {
+ super(InterTaskDataStoreProperty.class);
}
@Override
@@ -45,9 +45,9 @@ public final class SkewDataStorePass extends AnnotatingPass {
dag.getIncomingEdgesOf(v).forEach(edge -> {
// we want it to be in the same stage
if (edge.equals(edgeToUseMemory)) {
- edge.setProperty(DataStoreProperty.of(DataStoreProperty.Value.MemoryStore));
+ edge.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.MemoryStore));
} else {
- edge.setProperty(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore));
+ edge.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.LocalFileStore));
}
});
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SkewMetricCollectionPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DataSkewEdgeMetricCollectionPass.java
similarity index 81%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SkewMetricCollectionPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DataSkewEdgeMetricCollectionPass.java
index fe19942..614d81b 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SkewMetricCollectionPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DataSkewEdgeMetricCollectionPass.java
@@ -17,7 +17,7 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.MetricCollectionBarrierVertex;
import edu.snu.nemo.common.ir.edge.executionproperty.MetricCollectionProperty;
@@ -29,12 +29,12 @@ import java.util.Collections;
* It specifies the outgoing Shuffle edges from MetricCollectionVertices with a MetricCollection ExecutionProperty
* which lets the edge to know what metric collection it should perform.
*/
-public final class SkewMetricCollectionPass extends AnnotatingPass {
+public final class DataSkewEdgeMetricCollectionPass extends AnnotatingPass {
/**
* Default constructor.
*/
- public SkewMetricCollectionPass() {
- super(MetricCollectionProperty.class, Collections.singleton(CommunicationPatternProperty.class));
+ public DataSkewEdgeMetricCollectionPass() {
+ super(MetricCollectionProperty.class, Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -44,8 +44,8 @@ public final class SkewMetricCollectionPass extends AnnotatingPass {
if (v instanceof MetricCollectionBarrierVertex) {
dag.getOutgoingEdgesOf(v).forEach(edge -> {
// double checking.
- if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
- .equals(CommunicationPatternProperty.Value.Shuffle)) {
+ if (edge.getPropertyValue(DataCommunicationPatternProperty.class).get()
+ .equals(DataCommunicationPatternProperty.Value.Shuffle)) {
edge.setProperty(MetricCollectionProperty.of(MetricCollectionProperty.Value.DataSkewRuntimePass));
}
});
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SkewPartitionerPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DataSkewEdgePartitionerPass.java
similarity index 91%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SkewPartitionerPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DataSkewEdgePartitionerPass.java
index 2b9b903..4fcfe41 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SkewPartitionerPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DataSkewEdgePartitionerPass.java
@@ -26,13 +26,13 @@ import java.util.Collections;
import java.util.List;
/**
- * Transient resource pass for tagging edges with {@link PartitionerProperty}.
+ * Pado pass for tagging edges with {@link PartitionerProperty}.
*/
-public final class SkewPartitionerPass extends AnnotatingPass {
+public final class DataSkewEdgePartitionerPass extends AnnotatingPass {
/**
* Default constructor.
*/
- public SkewPartitionerPass() {
+ public DataSkewEdgePartitionerPass() {
super(PartitionerProperty.class, Collections.singleton(MetricCollectionProperty.class));
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SkewResourceSkewedDataPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DataSkewVertexPass.java
similarity index 84%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SkewResourceSkewedDataPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DataSkewVertexPass.java
index ab2f4be..2f543cb 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SkewResourceSkewedDataPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DataSkewVertexPass.java
@@ -20,7 +20,7 @@ import edu.snu.nemo.common.ir.edge.IREdge;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.MetricCollectionBarrierVertex;
import edu.snu.nemo.common.ir.vertex.executionproperty.DynamicOptimizationProperty;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceSkewedDataProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.SkewnessAwareSchedulingProperty;
import java.util.List;
@@ -28,11 +28,11 @@ import java.util.List;
* Pass to annotate the DAG for a job to perform data skew.
* It specifies which optimization to perform on the MetricCollectionBarrierVertex.
*/
-public final class SkewResourceSkewedDataPass extends AnnotatingPass {
+public final class DataSkewVertexPass extends AnnotatingPass {
/**
* Default constructor.
*/
- public SkewResourceSkewedDataPass() {
+ public DataSkewVertexPass() {
super(DynamicOptimizationProperty.class);
}
@@ -55,8 +55,8 @@ public final class SkewResourceSkewedDataPass extends AnnotatingPass {
.of(DynamicOptimizationProperty.Value.DataSkewRuntimePass)));
dag.getVertices().stream()
.filter(v -> hasMetricCollectionBarrierVertexAsParent(dag, v)
- && !v.getExecutionProperties().containsKey(ResourceSkewedDataProperty.class))
- .forEach(v -> v.getExecutionProperties().put(ResourceSkewedDataProperty.of(true)));
+ && !v.getExecutionProperties().containsKey(SkewnessAwareSchedulingProperty.class))
+ .forEach(v -> v.getExecutionProperties().put(SkewnessAwareSchedulingProperty.of(true)));
return dag;
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataPersistencePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeUsedDataHandlingPass.java
similarity index 52%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataPersistencePass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeUsedDataHandlingPass.java
index 8f96da5..0ea2d13 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataPersistencePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultEdgeUsedDataHandlingPass.java
@@ -17,36 +17,36 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataPersistenceProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.UsedDataHandlingProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import java.util.Collections;
/**
- * Pass for initiating IREdge data persistence ExecutionProperty with default values.
+ * Pass for initiating IREdge UsedDataHandling ExecutionProperty with default values.
*/
-public final class DefaultDataPersistencePass extends AnnotatingPass {
+public final class DefaultEdgeUsedDataHandlingPass extends AnnotatingPass {
/**
* Default constructor.
*/
- public DefaultDataPersistencePass() {
- super(DataPersistenceProperty.class, Collections.singleton(DataStoreProperty.class));
+ public DefaultEdgeUsedDataHandlingPass() {
+ super(UsedDataHandlingProperty.class, Collections.singleton(InterTaskDataStoreProperty.class));
}
@Override
public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
dag.topologicalDo(irVertex ->
dag.getIncomingEdgesOf(irVertex).forEach(irEdge -> {
- if (!irEdge.getPropertyValue(DataPersistenceProperty.class).isPresent()) {
- final DataStoreProperty.Value dataStoreValue
- = irEdge.getPropertyValue(DataStoreProperty.class).get();
- if (DataStoreProperty.Value.MemoryStore.equals(dataStoreValue)
- || DataStoreProperty.Value.SerializedMemoryStore.equals(dataStoreValue)) {
- irEdge.setProperty(DataPersistenceProperty.of(DataPersistenceProperty.Value.Discard));
+ if (!irEdge.getPropertyValue(UsedDataHandlingProperty.class).isPresent()) {
+ final InterTaskDataStoreProperty.Value dataStoreValue
+ = irEdge.getPropertyValue(InterTaskDataStoreProperty.class).get();
+ if (InterTaskDataStoreProperty.Value.MemoryStore.equals(dataStoreValue)
+ || InterTaskDataStoreProperty.Value.SerializedMemoryStore.equals(dataStoreValue)) {
+ irEdge.setProperty(UsedDataHandlingProperty.of(UsedDataHandlingProperty.Value.Discard));
} else {
- irEdge.setProperty(DataPersistenceProperty.of(DataPersistenceProperty.Value.Keep));
+ irEdge.setProperty(UsedDataHandlingProperty.of(UsedDataHandlingProperty.Value.Keep));
}
}
}));
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataStorePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultInterTaskDataStorePass.java
similarity index 75%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataStorePass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultInterTaskDataStorePass.java
index 6cdb126..3e849d8 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultDataStorePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultInterTaskDataStorePass.java
@@ -17,7 +17,7 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import java.util.Collections;
@@ -25,21 +25,21 @@ import java.util.Collections;
/**
* Edge data store pass to process inter-stage memory store edges.
*/
-public final class DefaultDataStorePass extends AnnotatingPass {
+public final class DefaultInterTaskDataStorePass extends AnnotatingPass {
/**
* Default constructor.
*/
- public DefaultDataStorePass() {
- super(DataStoreProperty.class, Collections.emptySet());
+ public DefaultInterTaskDataStorePass() {
+ super(InterTaskDataStoreProperty.class, Collections.emptySet());
}
@Override
public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
dag.getVertices().forEach(vertex -> {
dag.getIncomingEdgesOf(vertex).stream()
- .filter(edge -> !edge.getExecutionProperties().containsKey(DataStoreProperty.class))
+ .filter(edge -> !edge.getExecutionProperties().containsKey(InterTaskDataStoreProperty.class))
.forEach(edge -> edge.setProperty(
- DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore)));
+ InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.LocalFileStore)));
});
return dag;
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultParallelismPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultParallelismPass.java
index 8b85454..d26320f 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultParallelismPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultParallelismPass.java
@@ -17,7 +17,7 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.SourceVertex;
import edu.snu.nemo.common.dag.DAG;
@@ -49,7 +49,7 @@ public final class DefaultParallelismPass extends AnnotatingPass {
*/
public DefaultParallelismPass(final int desiredSourceParallelism,
final int shuffleDecreaseFactor) {
- super(ParallelismProperty.class, Collections.singleton(CommunicationPatternProperty.class));
+ super(ParallelismProperty.class, Collections.singleton(DataCommunicationPatternProperty.class));
this.desiredSourceParallelism = desiredSourceParallelism;
this.shuffleDecreaseFactor = shuffleDecreaseFactor;
}
@@ -75,13 +75,13 @@ public final class DefaultParallelismPass extends AnnotatingPass {
// No reason to propagate via Broadcast edges, as the data streams that will use the broadcasted data
// as a sideInput will have their own number of parallelism
final Integer o2oParallelism = inEdges.stream()
- .filter(edge -> CommunicationPatternProperty.Value.OneToOne
- .equals(edge.getPropertyValue(CommunicationPatternProperty.class).get()))
+ .filter(edge -> DataCommunicationPatternProperty.Value.OneToOne
+ .equals(edge.getPropertyValue(DataCommunicationPatternProperty.class).get()))
.mapToInt(edge -> edge.getSrc().getPropertyValue(ParallelismProperty.class).get())
.max().orElse(1);
final Integer shuffleParallelism = inEdges.stream()
- .filter(edge -> CommunicationPatternProperty.Value.Shuffle
- .equals(edge.getPropertyValue(CommunicationPatternProperty.class).get()))
+ .filter(edge -> DataCommunicationPatternProperty.Value.Shuffle
+ .equals(edge.getPropertyValue(DataCommunicationPatternProperty.class).get()))
.mapToInt(edge -> edge.getSrc().getPropertyValue(ParallelismProperty.class).get())
.map(i -> i / shuffleDecreaseFactor)
.max().orElse(1);
@@ -113,8 +113,8 @@ public final class DefaultParallelismPass extends AnnotatingPass {
final Integer parallelism) {
final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
final Integer ancestorParallelism = inEdges.stream()
- .filter(edge -> CommunicationPatternProperty.Value.OneToOne
- .equals(edge.getPropertyValue(CommunicationPatternProperty.class).get()))
+ .filter(edge -> DataCommunicationPatternProperty.Value.OneToOne
+ .equals(edge.getPropertyValue(DataCommunicationPatternProperty.class).get()))
.map(IREdge::getSrc)
.mapToInt(inVertex -> recursivelySynchronizeO2OParallelism(dag, inVertex, parallelism))
.max().orElse(1);
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPass.java
index 190a552..ca788bd 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPass.java
@@ -20,9 +20,9 @@ import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.dag.Edge;
import edu.snu.nemo.common.dag.Vertex;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
import edu.snu.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty;
import org.apache.commons.lang3.mutable.MutableInt;
@@ -72,8 +72,8 @@ public final class DefaultScheduleGroupPass extends AnnotatingPass {
final boolean allowShuffleWithinScheduleGroup,
final boolean allowMultipleInEdgesWithinScheduleGroup) {
super(ScheduleGroupProperty.class, Stream.of(
- CommunicationPatternProperty.class,
- DataFlowProperty.class
+ DataCommunicationPatternProperty.class,
+ DataFlowModelProperty.class
).collect(Collectors.toSet()));
this.allowBroadcastWithinScheduleGroup = allowBroadcastWithinScheduleGroup;
this.allowShuffleWithinScheduleGroup = allowShuffleWithinScheduleGroup;
@@ -121,9 +121,9 @@ public final class DefaultScheduleGroupPass extends AnnotatingPass {
// Get ScheduleGroup(s) that push data to the connectedIRVertex
final Set<ScheduleGroup> pushScheduleGroups = new HashSet<>();
for (final IREdge edgeToConnectedIRVertex : dag.getIncomingEdgesOf(connectedIRVertex)) {
- if (edgeToConnectedIRVertex.getPropertyValue(DataFlowProperty.class)
- .orElseThrow(() -> new RuntimeException(String.format("DataFlowProperty for %s must be set",
- edgeToConnectedIRVertex.getId()))) == DataFlowProperty.Value.Push) {
+ if (edgeToConnectedIRVertex.getPropertyValue(DataFlowModelProperty.class)
+ .orElseThrow(() -> new RuntimeException(String.format("DataFlowModelProperty for %s must be set",
+ edgeToConnectedIRVertex.getId()))) == DataFlowModelProperty.Value.Push) {
pushScheduleGroups.add(irVertexToScheduleGroupMap.get(edgeToConnectedIRVertex.getSrc()));
}
}
@@ -142,16 +142,16 @@ public final class DefaultScheduleGroupPass extends AnnotatingPass {
// new ScheduleGroup
mergability = false;
}
- final CommunicationPatternProperty.Value communicationPattern = edgeToConnectedIRVertex
- .getPropertyValue(CommunicationPatternProperty.class).orElseThrow(
- () -> new RuntimeException(String.format("CommunicationPatternProperty for %s must be set",
+ final DataCommunicationPatternProperty.Value communicationPattern = edgeToConnectedIRVertex
+ .getPropertyValue(DataCommunicationPatternProperty.class).orElseThrow(
+ () -> new RuntimeException(String.format("DataCommunicationPatternProperty for %s must be set",
edgeToConnectedIRVertex.getId())));
if (!allowBroadcastWithinScheduleGroup
- && communicationPattern == CommunicationPatternProperty.Value.BroadCast) {
+ && communicationPattern == DataCommunicationPatternProperty.Value.BroadCast) {
mergability = false;
}
if (!allowShuffleWithinScheduleGroup
- && communicationPattern == CommunicationPatternProperty.Value.Shuffle) {
+ && communicationPattern == DataCommunicationPatternProperty.Value.Shuffle) {
mergability = false;
}
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DisaggregationEdgeDataStorePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DisaggregationEdgeDataStorePass.java
index e498c74..76983fd 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DisaggregationEdgeDataStorePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DisaggregationEdgeDataStorePass.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.dag.DAG;
@@ -32,7 +32,7 @@ public final class DisaggregationEdgeDataStorePass extends AnnotatingPass {
* Default constructor.
*/
public DisaggregationEdgeDataStorePass() {
- super(DataStoreProperty.class, Collections.singleton(DataStoreProperty.class));
+ super(InterTaskDataStoreProperty.class, Collections.singleton(InterTaskDataStoreProperty.class));
}
@Override
@@ -40,7 +40,7 @@ public final class DisaggregationEdgeDataStorePass extends AnnotatingPass {
dag.getVertices().forEach(vertex -> { // Initialize the DataStore of the DAG with GlusterFileStore.
final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
inEdges.forEach(edge -> {
- edge.setProperty(DataStoreProperty.of(DataStoreProperty.Value.GlusterFileStore));
+ edge.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.GlusterFileStore));
});
});
return dag;
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSlotPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ExecutorSlotCompliancePass.java
similarity index 62%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSlotPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ExecutorSlotCompliancePass.java
index 966fb50..e1b2123 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSlotPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ExecutorSlotCompliancePass.java
@@ -18,23 +18,23 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
import edu.snu.nemo.common.ir.vertex.IRVertex;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorSlotComplianceProperty;
/**
- * Sets {@link ResourceSlotProperty}.
+ * Sets {@link ExecutorSlotComplianceProperty}.
*/
-public final class ResourceSlotPass extends AnnotatingPass {
+public final class ExecutorSlotCompliancePass extends AnnotatingPass {
- public ResourceSlotPass() {
- super(ResourceSlotProperty.class);
+ public ExecutorSlotCompliancePass() {
+ super(ExecutorSlotComplianceProperty.class);
}
@Override
public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
- // On every vertex, if ResourceSlotProperty is not set, put it as true.
+ // On every vertex, if ExecutorSlotComplianceProperty is not set, put it as true.
dag.getVertices().stream()
- .filter(v -> !v.getExecutionProperties().containsKey(ResourceSlotProperty.class))
- .forEach(v -> v.getExecutionProperties().put(ResourceSlotProperty.of(true)));
+ .filter(v -> !v.getExecutionProperties().containsKey(ExecutorSlotComplianceProperty.class))
+ .forEach(v -> v.getExecutionProperties().put(ExecutorSlotComplianceProperty.of(true)));
return dag;
}
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSitePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/NodeNamesAssignmentPass.java
similarity index 89%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSitePass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/NodeNamesAssignmentPass.java
index 6189826..f86d355 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceSitePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/NodeNamesAssignmentPass.java
@@ -19,9 +19,9 @@ import com.fasterxml.jackson.core.TreeNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceSiteProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.NodeNamesProperty;
import edu.snu.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
import org.apache.commons.math3.optim.BaseOptimizer;
import org.apache.commons.math3.optim.PointValuePair;
@@ -38,23 +38,23 @@ import java.util.*;
/**
* Computes and assigns appropriate share of nodes to each irVertex to minimize shuffle time,
* with respect to bandwidth restrictions of nodes. If bandwidth information is not given, this pass does nothing.
- * This pass optimizes task assignment considering nonuniform network bandwidths between resources.
- * Ref. http://pages.cs.wisc.edu/~akella/papers/gda-sigcomm15.pdf
+ * This pass follows task assignment of Iridium-style optimization.
+ * http://pages.cs.wisc.edu/~akella/papers/gda-sigcomm15.pdf
*
* <h3>Assumptions</h3>
* This pass assumes no skew in input or intermediate data, so that the number of Task assigned to a node
* is proportional to the data size handled by the node.
- * Also, this pass assumes stages with empty map as {@link ResourceSiteProperty} are assigned to nodes evenly.
+ * Also, this pass assumes stages with empty map as {@link NodeNamesProperty} are assigned to nodes evenly.
* For example, if source splits are not distributed evenly, any source location-aware scheduling policy will
* assign TaskGroups unevenly.
* Also, this pass assumes network bandwidth to be the bottleneck. Each node should have enough capacity to run
* TaskGroups immediately as scheduler attempts to schedule a TaskGroup.
*/
-public final class ResourceSitePass extends AnnotatingPass {
+public final class NodeNamesAssignmentPass extends AnnotatingPass {
// Index of the objective parameter, in the coefficient vector
private static final int OBJECTIVE_COEFFICIENT_INDEX = 0;
- private static final Logger LOG = LoggerFactory.getLogger(ResourceSitePass.class);
+ private static final Logger LOG = LoggerFactory.getLogger(NodeNamesAssignmentPass.class);
private static final HashMap<String, Integer> EMPTY_MAP = new HashMap<>();
private static String bandwidthSpecificationString = "";
@@ -63,14 +63,14 @@ public final class ResourceSitePass extends AnnotatingPass {
/**
* Default constructor.
*/
- public ResourceSitePass() {
- super(ResourceSiteProperty.class, Collections.singleton(ParallelismProperty.class));
+ public NodeNamesAssignmentPass() {
+ super(NodeNamesProperty.class, Collections.singleton(ParallelismProperty.class));
}
@Override
public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
if (bandwidthSpecificationString.isEmpty()) {
- dag.topologicalDo(irVertex -> irVertex.setProperty(ResourceSiteProperty.of(EMPTY_MAP)));
+ dag.topologicalDo(irVertex -> irVertex.setProperty(NodeNamesProperty.of(EMPTY_MAP)));
} else {
assignNodeShares(dag, BandwidthSpecification.fromJsonString(bandwidthSpecificationString));
}
@@ -101,17 +101,17 @@ public final class ResourceSitePass extends AnnotatingPass {
if (inEdges.size() == 0) {
// This vertex is root vertex.
// Fall back to setting even distribution
- irVertex.getExecutionProperties().put(ResourceSiteProperty.of(EMPTY_MAP));
+ irVertex.getExecutionProperties().put(NodeNamesProperty.of(EMPTY_MAP));
} else if (isOneToOneEdge(inEdges)) {
final Optional<HashMap<String, Integer>> property = inEdges.iterator().next().getSrc()
- .getExecutionProperties().get(ResourceSiteProperty.class);
- irVertex.getExecutionProperties().put(ResourceSiteProperty.of(property.get()));
+ .getExecutionProperties().get(NodeNamesProperty.class);
+ irVertex.getExecutionProperties().put(NodeNamesProperty.of(property.get()));
} else {
// This IRVertex has shuffle inEdge(s), or has multiple inEdges.
final Map<String, Integer> parentLocationShares = new HashMap<>();
for (final IREdge edgeToIRVertex : dag.getIncomingEdgesOf(irVertex)) {
final IRVertex parentVertex = edgeToIRVertex.getSrc();
- final Map<String, Integer> parentShares = parentVertex.getPropertyValue(ResourceSiteProperty.class).get();
+ final Map<String, Integer> parentShares = parentVertex.getPropertyValue(NodeNamesProperty.class).get();
final int parentParallelism = parentVertex.getPropertyValue(ParallelismProperty.class)
.orElseThrow(() -> new RuntimeException("Parallelism property required"));
final Map<String, Integer> shares = parentShares.isEmpty() ? getEvenShares(bandwidthSpecification.getNodes(),
@@ -135,7 +135,7 @@ public final class ResourceSitePass extends AnnotatingPass {
shares.put(nodeName, shares.get(nodeName) + 1);
remainder--;
}
- irVertex.getExecutionProperties().put(ResourceSiteProperty.of(shares));
+ irVertex.getExecutionProperties().put(NodeNamesProperty.of(shares));
}
});
}
@@ -146,8 +146,8 @@ public final class ResourceSitePass extends AnnotatingPass {
*/
private static boolean isOneToOneEdge(final Collection<IREdge> inEdges) {
return inEdges.size() == 1 && inEdges.iterator().next()
- .getPropertyValue(CommunicationPatternProperty.class).get()
- .equals(CommunicationPatternProperty.Value.OneToOne);
+ .getPropertyValue(DataCommunicationPatternProperty.class).get()
+ .equals(DataCommunicationPatternProperty.Value.OneToOne);
}
/**
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataFlowPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/PadoEdgeDataFlowModelPass.java
similarity index 67%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataFlowPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/PadoEdgeDataFlowModelPass.java
index 3433dbe..b74a995 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataFlowPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/PadoEdgeDataFlowModelPass.java
@@ -17,24 +17,24 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
import java.util.Collections;
import java.util.List;
-import static edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.TransientResourceDataStorePass.fromTransientToReserved;
+import static edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.PadoEdgeDataStorePass.fromTransientToReserved;
/**
- * Push from transient resources to reserved resources.
+ * Pado pass for tagging edges with DataFlowModel ExecutionProperty.
*/
-public final class TransientResourceDataFlowPass extends AnnotatingPass {
+public final class PadoEdgeDataFlowModelPass extends AnnotatingPass {
/**
* Default constructor.
*/
- public TransientResourceDataFlowPass() {
- super(DataFlowProperty.class, Collections.singleton(ResourcePriorityProperty.class));
+ public PadoEdgeDataFlowModelPass() {
+ super(DataFlowModelProperty.class, Collections.singleton(ExecutorPlacementProperty.class));
}
@Override
@@ -44,9 +44,9 @@ public final class TransientResourceDataFlowPass extends AnnotatingPass {
if (!inEdges.isEmpty()) {
inEdges.forEach(edge -> {
if (fromTransientToReserved(edge)) {
- edge.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Push));
+ edge.setProperty(DataFlowModelProperty.of(DataFlowModelProperty.Value.Push));
} else {
- edge.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Pull));
+ edge.setProperty(DataFlowModelProperty.of(DataFlowModelProperty.Value.Pull));
}
});
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataStorePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/PadoEdgeDataStorePass.java
similarity index 54%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataStorePass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/PadoEdgeDataStorePass.java
index bcf0d0e..8e4023c 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourceDataStorePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/PadoEdgeDataStorePass.java
@@ -16,24 +16,24 @@
package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.dag.DAG;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
import java.util.Collections;
import java.util.List;
/**
- * Transient resource pass for tagging edges with DataStore ExecutionProperty.
+ * Pado pass for tagging edges with DataStore ExecutionProperty.
*/
-public final class TransientResourceDataStorePass extends AnnotatingPass {
+public final class PadoEdgeDataStorePass extends AnnotatingPass {
/**
* Default constructor.
*/
- public TransientResourceDataStorePass() {
- super(DataStoreProperty.class, Collections.singleton(ResourcePriorityProperty.class));
+ public PadoEdgeDataStorePass() {
+ super(InterTaskDataStoreProperty.class, Collections.singleton(ExecutorPlacementProperty.class));
}
@Override
@@ -43,12 +43,12 @@ public final class TransientResourceDataStorePass extends AnnotatingPass {
if (!inEdges.isEmpty()) {
inEdges.forEach(edge -> {
if (fromTransientToReserved(edge) || fromReservedToTransient(edge)) {
- edge.setProperty(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore));
- } else if (CommunicationPatternProperty.Value.OneToOne
- .equals(edge.getPropertyValue(CommunicationPatternProperty.class).get())) {
- edge.setProperty(DataStoreProperty.of(DataStoreProperty.Value.MemoryStore));
+ edge.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.LocalFileStore));
+ } else if (DataCommunicationPatternProperty.Value.OneToOne
+ .equals(edge.getPropertyValue(DataCommunicationPatternProperty.class).get())) {
+ edge.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.MemoryStore));
} else {
- edge.setProperty(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore));
+ edge.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.LocalFileStore));
}
});
}
@@ -62,10 +62,10 @@ public final class TransientResourceDataStorePass extends AnnotatingPass {
* @return whether or not the edge satisfies the condition.
*/
static boolean fromTransientToReserved(final IREdge irEdge) {
- return ResourcePriorityProperty.TRANSIENT
- .equals(irEdge.getSrc().getPropertyValue(ResourcePriorityProperty.class).get())
- && ResourcePriorityProperty.RESERVED
- .equals(irEdge.getDst().getPropertyValue(ResourcePriorityProperty.class).get());
+ return ExecutorPlacementProperty.TRANSIENT
+ .equals(irEdge.getSrc().getPropertyValue(ExecutorPlacementProperty.class).get())
+ && ExecutorPlacementProperty.RESERVED
+ .equals(irEdge.getDst().getPropertyValue(ExecutorPlacementProperty.class).get());
}
/**
@@ -74,9 +74,9 @@ public final class TransientResourceDataStorePass extends AnnotatingPass {
* @return whether or not the edge satisfies the condition.
*/
static boolean fromReservedToTransient(final IREdge irEdge) {
- return ResourcePriorityProperty.RESERVED
- .equals(irEdge.getSrc().getPropertyValue(ResourcePriorityProperty.class).get())
- && ResourcePriorityProperty.TRANSIENT
- .equals(irEdge.getDst().getPropertyValue(ResourcePriorityProperty.class).get());
+ return ExecutorPlacementProperty.RESERVED
+ .equals(irEdge.getSrc().getPropertyValue(ExecutorPlacementProperty.class).get())
+ && ExecutorPlacementProperty.TRANSIENT
+ .equals(irEdge.getDst().getPropertyValue(ExecutorPlacementProperty.class).get());
}
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourcePriorityPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/PadoVertexExecutorPlacementPass.java
similarity index 60%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourcePriorityPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/PadoVertexExecutorPlacementPass.java
index fd1eb5d..64e4dfb 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/TransientResourcePriorityPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/PadoVertexExecutorPlacementPass.java
@@ -16,23 +16,23 @@
package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.dag.DAG;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
import java.util.Collections;
import java.util.List;
/**
- * Place valuable computations on reserved resources, and the rest on transient resources.
+ * Pado pass for tagging vertices.
*/
-public final class TransientResourcePriorityPass extends AnnotatingPass {
+public final class PadoVertexExecutorPlacementPass extends AnnotatingPass {
/**
* Default constructor.
*/
- public TransientResourcePriorityPass() {
- super(ResourcePriorityProperty.class, Collections.singleton(CommunicationPatternProperty.class));
+ public PadoVertexExecutorPlacementPass() {
+ super(ExecutorPlacementProperty.class, Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -40,12 +40,12 @@ public final class TransientResourcePriorityPass extends AnnotatingPass {
dag.topologicalDo(vertex -> {
final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
if (inEdges.isEmpty()) {
- vertex.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.TRANSIENT));
+ vertex.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.TRANSIENT));
} else {
if (hasM2M(inEdges) || allO2OFromReserved(inEdges)) {
- vertex.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.RESERVED));
+ vertex.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.RESERVED));
} else {
- vertex.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.TRANSIENT));
+ vertex.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.TRANSIENT));
}
}
});
@@ -59,8 +59,8 @@ public final class TransientResourcePriorityPass extends AnnotatingPass {
*/
private boolean hasM2M(final List<IREdge> irEdges) {
return irEdges.stream().anyMatch(edge ->
- edge.getPropertyValue(CommunicationPatternProperty.class).get()
- .equals(CommunicationPatternProperty.Value.Shuffle));
+ edge.getPropertyValue(DataCommunicationPatternProperty.class).get()
+ .equals(DataCommunicationPatternProperty.Value.Shuffle));
}
/**
@@ -70,9 +70,9 @@ public final class TransientResourcePriorityPass extends AnnotatingPass {
*/
private boolean allO2OFromReserved(final List<IREdge> irEdges) {
return irEdges.stream()
- .allMatch(edge -> CommunicationPatternProperty.Value.OneToOne.equals(
- edge.getPropertyValue(CommunicationPatternProperty.class).get())
- && edge.getSrc().getPropertyValue(ResourcePriorityProperty.class).get().equals(
- ResourcePriorityProperty.RESERVED));
+ .allMatch(edge -> DataCommunicationPatternProperty.Value.OneToOne.equals(
+ edge.getPropertyValue(DataCommunicationPatternProperty.class).get())
+ && edge.getSrc().getPropertyValue(ExecutorPlacementProperty.class).get().equals(
+ ExecutorPlacementProperty.RESERVED));
}
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataFlowPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeDataFlowModelPass.java
similarity index 61%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataFlowPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeDataFlowModelPass.java
index 16a3b81..95154b4 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataFlowPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeDataFlowModelPass.java
@@ -17,23 +17,23 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
import java.util.Collections;
import java.util.List;
/**
- * A pass to optimize large shuffle by tagging edges.
+ * A pass to support Sailfish-like shuffle by tagging edges.
* This pass handles the DataFlowModel ExecutionProperty.
*/
-public final class LargeShuffleDataFlowPass extends AnnotatingPass {
+public final class SailfishEdgeDataFlowModelPass extends AnnotatingPass {
/**
* Default constructor.
*/
- public LargeShuffleDataFlowPass() {
- super(DataFlowProperty.class, Collections.singleton(CommunicationPatternProperty.class));
+ public SailfishEdgeDataFlowModelPass() {
+ super(DataFlowModelProperty.class, Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -41,11 +41,11 @@ public final class LargeShuffleDataFlowPass extends AnnotatingPass {
dag.getVertices().forEach(vertex -> {
final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
inEdges.forEach(edge -> {
- if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
- .equals(CommunicationPatternProperty.Value.Shuffle)) {
- edge.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Push)); // Push to the merger vertex.
+ if (edge.getPropertyValue(DataCommunicationPatternProperty.class).get()
+ .equals(DataCommunicationPatternProperty.Value.Shuffle)) {
+ edge.setProperty(DataFlowModelProperty.of(DataFlowModelProperty.Value.Push)); // Push to the merger vertex.
} else {
- edge.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Pull));
+ edge.setProperty(DataFlowModelProperty.of(DataFlowModelProperty.Value.Pull));
}
});
});
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataStorePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeDataStorePass.java
similarity index 60%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataStorePass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeDataStorePass.java
index c80f675..fe5d996 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataStorePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeDataStorePass.java
@@ -17,22 +17,22 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import java.util.Collections;
/**
- * A pass to optimize large shuffle by tagging edges.
+ * A pass to support Sailfish-like shuffle by tagging edges.
* This pass handles the DataStore ExecutionProperty.
*/
-public final class LargeShuffleDataStorePass extends AnnotatingPass {
+public final class SailfishEdgeDataStorePass extends AnnotatingPass {
/**
* Default constructor.
*/
- public LargeShuffleDataStorePass() {
- super(DataStoreProperty.class, Collections.singleton(CommunicationPatternProperty.class));
+ public SailfishEdgeDataStorePass() {
+ super(InterTaskDataStoreProperty.class, Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -40,19 +40,19 @@ public final class LargeShuffleDataStorePass extends AnnotatingPass {
dag.getVertices().forEach(vertex -> {
// Find the merger vertex inserted by reshaping pass.
if (dag.getIncomingEdgesOf(vertex).stream().anyMatch(irEdge ->
- CommunicationPatternProperty.Value.Shuffle
- .equals(irEdge.getPropertyValue(CommunicationPatternProperty.class).get()))) {
+ DataCommunicationPatternProperty.Value.Shuffle
+ .equals(irEdge.getPropertyValue(DataCommunicationPatternProperty.class).get()))) {
dag.getIncomingEdgesOf(vertex).forEach(edgeToMerger -> {
- if (CommunicationPatternProperty.Value.Shuffle
- .equals(edgeToMerger.getPropertyValue(CommunicationPatternProperty.class).get())) {
+ if (DataCommunicationPatternProperty.Value.Shuffle
+ .equals(edgeToMerger.getPropertyValue(DataCommunicationPatternProperty.class).get())) {
// Pass data through memory to the merger vertex.
- edgeToMerger.setProperty(DataStoreProperty
- .of(DataStoreProperty.Value.SerializedMemoryStore));
+ edgeToMerger.setProperty(InterTaskDataStoreProperty
+ .of(InterTaskDataStoreProperty.Value.SerializedMemoryStore));
}
});
dag.getOutgoingEdgesOf(vertex).forEach(edgeFromMerger ->
// Merge the input data and write it immediately to the remote disk.
- edgeFromMerger.setProperty(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore)));
+ edgeFromMerger.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.LocalFileStore)));
}
});
return dag;
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDecoderPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeDecoderPass.java
similarity index 75%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDecoderPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeDecoderPass.java
index ed66bcd..fc8b7a3 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDecoderPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeDecoderPass.java
@@ -18,7 +18,7 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.coder.BytesDecoderFactory;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DecoderProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
@@ -26,16 +26,16 @@ import java.util.Collections;
import java.util.List;
/**
- * A pass to optimize large shuffle by tagging edges.
+ * A pass to support Sailfish-like shuffle by tagging edges.
* This pass modifies the decoder property toward {@link edu.snu.nemo.common.ir.vertex.transform.RelayTransform}
* to read data as byte arrays.
*/
-public final class LargeShuffleDecoderPass extends AnnotatingPass {
+public final class SailfishEdgeDecoderPass extends AnnotatingPass {
/**
* Default constructor.
*/
- public LargeShuffleDecoderPass() {
- super(DecoderProperty.class, Collections.singleton(CommunicationPatternProperty.class));
+ public SailfishEdgeDecoderPass() {
+ super(DecoderProperty.class, Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -43,8 +43,8 @@ public final class LargeShuffleDecoderPass extends AnnotatingPass {
dag.getVertices().forEach(vertex -> {
final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
inEdges.forEach(edge -> {
- if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
- .equals(CommunicationPatternProperty.Value.Shuffle)) {
+ if (edge.getPropertyValue(DataCommunicationPatternProperty.class).get()
+ .equals(DataCommunicationPatternProperty.Value.Shuffle)) {
edge.setProperty(DecoderProperty.of(BytesDecoderFactory.of()));
}
});
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleEncoderPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeEncoderPass.java
similarity index 77%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleEncoderPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeEncoderPass.java
index 8cad7b0..418b9af 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleEncoderPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeEncoderPass.java
@@ -18,7 +18,7 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.coder.BytesEncoderFactory;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.EncoderProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
@@ -26,16 +26,16 @@ import java.util.Collections;
import java.util.List;
/**
- * A pass to optimize large shuffle by tagging edges.
+ * A pass to support Sailfish-like shuffle by tagging edges.
* This pass modifies the encoder property toward {@link edu.snu.nemo.common.ir.vertex.transform.RelayTransform}
* to write data as byte arrays.
*/
-public final class LargeShuffleEncoderPass extends AnnotatingPass {
+public final class SailfishEdgeEncoderPass extends AnnotatingPass {
/**
* Default constructor.
*/
- public LargeShuffleEncoderPass() {
- super(EncoderProperty.class, Collections.singleton(CommunicationPatternProperty.class));
+ public SailfishEdgeEncoderPass() {
+ super(EncoderProperty.class, Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -43,8 +43,8 @@ public final class LargeShuffleEncoderPass extends AnnotatingPass {
dag.getVertices().forEach(vertex -> {
final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
inEdges.forEach(edge -> {
- if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
- .equals(CommunicationPatternProperty.Value.Shuffle)) {
+ if (edge.getPropertyValue(DataCommunicationPatternProperty.class).get()
+ .equals(DataCommunicationPatternProperty.Value.Shuffle)) {
dag.getOutgoingEdgesOf(edge.getDst())
.forEach(edgeFromRelay -> {
edgeFromRelay.setProperty(EncoderProperty.of(BytesEncoderFactory.of()));
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataPersistencePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeUsedDataHandlingPass.java
similarity index 58%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataPersistencePass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeUsedDataHandlingPass.java
index 7ebad49..dd3a532 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleDataPersistencePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishEdgeUsedDataHandlingPass.java
@@ -17,32 +17,32 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataPersistenceProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.UsedDataHandlingProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import java.util.Collections;
/**
- * A pass to optimize large shuffle by tagging edges.
- * This pass handles the data persistence ExecutionProperty.
+ * A pass to support Sailfish-like shuffle by tagging edges.
+ * This pass handles the UsedDataHandling ExecutionProperty.
*/
-public final class LargeShuffleDataPersistencePass extends AnnotatingPass {
+public final class SailfishEdgeUsedDataHandlingPass extends AnnotatingPass {
/**
* Default constructor.
*/
- public LargeShuffleDataPersistencePass() {
- super(DataPersistenceProperty.class, Collections.singleton(DataFlowProperty.class));
+ public SailfishEdgeUsedDataHandlingPass() {
+ super(UsedDataHandlingProperty.class, Collections.singleton(DataFlowModelProperty.class));
}
@Override
public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
dag.topologicalDo(irVertex ->
dag.getIncomingEdgesOf(irVertex).forEach(irEdge -> {
- final DataFlowProperty.Value dataFlowModel = irEdge.getPropertyValue(DataFlowProperty.class).get();
- if (DataFlowProperty.Value.Push.equals(dataFlowModel)) {
- irEdge.setProperty(DataPersistenceProperty.of(DataPersistenceProperty.Value.Discard));
+ final DataFlowModelProperty.Value dataFlowModel = irEdge.getPropertyValue(DataFlowModelProperty.class).get();
+ if (DataFlowModelProperty.Value.Push.equals(dataFlowModel)) {
+ irEdge.setProperty(UsedDataHandlingProperty.of(UsedDataHandlingProperty.Value.Discard));
}
}));
return dag;
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleResourceSlotPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishVertexExecutorSlotCompliancePass.java
similarity index 51%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleResourceSlotPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishVertexExecutorSlotCompliancePass.java
index 388f7c2..4b18885 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/LargeShuffleResourceSlotPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SailfishVertexExecutorSlotCompliancePass.java
@@ -17,35 +17,35 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorSlotComplianceProperty;
import java.util.Collections;
/**
- * Sets {@link ResourceSlotProperty}.
+ * Sets {@link ExecutorSlotComplianceProperty}.
*/
-public final class LargeShuffleResourceSlotPass extends AnnotatingPass {
+public final class SailfishVertexExecutorSlotCompliancePass extends AnnotatingPass {
- public LargeShuffleResourceSlotPass() {
- super(ResourceSlotProperty.class, Collections.singleton(DataFlowProperty.class));
+ public SailfishVertexExecutorSlotCompliancePass() {
+ super(ExecutorSlotComplianceProperty.class, Collections.singleton(DataFlowModelProperty.class));
}
@Override
public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
- // On every vertex that receive push edge, if ResourceSlotProperty is not set, put it as false.
- // For other vertices, if ResourceSlotProperty is not set, put it as true.
+ // On every vertex that receive push edge, if ExecutorSlotComplianceProperty is not set, put it as false.
+ // For other vertices, if ExecutorSlotComplianceProperty is not set, put it as true.
dag.getVertices().stream()
- .filter(v -> !v.getExecutionProperties().containsKey(ResourceSlotProperty.class))
+ .filter(v -> !v.getExecutionProperties().containsKey(ExecutorSlotComplianceProperty.class))
.forEach(v -> {
if (dag.getIncomingEdgesOf(v).stream().anyMatch(
- e -> e.getPropertyValue(DataFlowProperty.class)
- .orElseThrow(() -> new RuntimeException(String.format("DataFlowProperty for %s must be set",
- e.getId()))).equals(DataFlowProperty.Value.Push))) {
- v.getExecutionProperties().put(ResourceSlotProperty.of(false));
+ e -> e.getPropertyValue(DataFlowModelProperty.class)
+ .orElseThrow(() -> new RuntimeException(String.format("DataFlowModelProperty for %s must be set",
+ e.getId()))).equals(DataFlowModelProperty.Value.Push))) {
+ v.getExecutionProperties().put(ExecutorSlotComplianceProperty.of(false));
} else {
- v.getExecutionProperties().put(ResourceSlotProperty.of(true));
+ v.getExecutionProperties().put(ExecutorSlotComplianceProperty.of(true));
}
});
return dag;
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ShuffleEdgePushPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ShuffleEdgePushPass.java
index feff84f..d548a1f 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ShuffleEdgePushPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ShuffleEdgePushPass.java
@@ -17,9 +17,9 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
import java.util.Collections;
import java.util.List;
@@ -33,7 +33,7 @@ public final class ShuffleEdgePushPass extends AnnotatingPass {
* Default constructor.
*/
public ShuffleEdgePushPass() {
- super(DataFlowProperty.class, Collections.singleton(CommunicationPatternProperty.class));
+ super(DataFlowModelProperty.class, Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -42,9 +42,9 @@ public final class ShuffleEdgePushPass extends AnnotatingPass {
final List<IREdge> inEdges = dag.getIncomingEdgesOf(vertex);
if (!inEdges.isEmpty()) {
inEdges.forEach(edge -> {
- if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
- .equals(CommunicationPatternProperty.Value.Shuffle)) {
- edge.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Push));
+ if (edge.getPropertyValue(DataCommunicationPatternProperty.class).get()
+ .equals(DataCommunicationPatternProperty.Value.Shuffle)) {
+ edge.setProperty(DataFlowModelProperty.of(DataFlowModelProperty.Value.Push));
}
});
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceLocalityPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SourceLocationAwareSchedulingPass.java
similarity index 60%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceLocalityPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SourceLocationAwareSchedulingPass.java
index 4862c7e..8c30d95 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/ResourceLocalityPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/SourceLocationAwareSchedulingPass.java
@@ -18,23 +18,23 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
import edu.snu.nemo.common.ir.vertex.IRVertex;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.SourceLocationAwareSchedulingProperty;
/**
- * Sets {@link ResourceLocalityProperty}.
+ * Sets {@link SourceLocationAwareSchedulingProperty}.
*/
-public final class ResourceLocalityPass extends AnnotatingPass {
+public final class SourceLocationAwareSchedulingPass extends AnnotatingPass {
- public ResourceLocalityPass() {
- super(ResourceLocalityProperty.class);
+ public SourceLocationAwareSchedulingPass() {
+ super(SourceLocationAwareSchedulingProperty.class);
}
@Override
public DAG<IRVertex, IREdge> apply(final DAG<IRVertex, IREdge> dag) {
- // On every vertex, if ResourceLocalityProperty is not set, put it as true.
+ // On every vertex, if SourceLocationAwareSchedulingProperty is not set, put it as true.
dag.getVertices().stream()
- .filter(v -> !v.getExecutionProperties().containsKey(ResourceLocalityProperty.class))
- .forEach(v -> v.getExecutionProperties().put(ResourceLocalityProperty.of(true)));
+ .filter(v -> !v.getExecutionProperties().containsKey(SourceLocationAwareSchedulingProperty.class))
+ .forEach(v -> v.getExecutionProperties().put(SourceLocationAwareSchedulingProperty.of(true)));
return dag;
}
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DataSkewCompositePass.java
similarity index 75%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DataSkewCompositePass.java
index 5e3d8fd..717ee1e 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DataSkewCompositePass.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.compiler.optimizer.pass.compiletime.composite;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.*;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.reshaping.SkewReshapingPass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.reshaping.DataSkewReshapingPass;
import java.util.Arrays;
@@ -24,21 +24,21 @@ import java.util.Arrays;
* Pass to modify the DAG for a job to perform data skew.
* It adds a {@link edu.snu.nemo.common.ir.vertex.MetricCollectionBarrierVertex} before Shuffle edges,
* to make a barrier before it, and to use the metrics to repartition the skewed data.
- * NOTE: we currently put the SkewCompositePass at the end of the list for each policies, as it needs to take a
+ * NOTE: we currently put the DataSkewCompositePass at the end of the list for each policies, as it needs to take a
* snapshot at the end of the pass. This could be prevented by modifying other passes to take the snapshot of the DAG
* at the end of each passes for metricCollectionVertices.
*/
-public final class SkewCompositePass extends CompositePass {
+public final class DataSkewCompositePass extends CompositePass {
/**
* Default constructor.
*/
- public SkewCompositePass() {
+ public DataSkewCompositePass() {
super(Arrays.asList(
- new SkewReshapingPass(),
- new SkewResourceSkewedDataPass(),
- new SkewDataStorePass(),
- new SkewMetricCollectionPass(),
- new SkewPartitionerPass()
+ new DataSkewReshapingPass(),
+ new DataSkewVertexPass(),
+ new DataSkewEdgeDataStorePass(),
+ new DataSkewEdgeMetricCollectionPass(),
+ new DataSkewEdgePartitionerPass()
));
}
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/TransientResourceCompositePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/PadoCompositePass.java
similarity index 68%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/TransientResourceCompositePass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/PadoCompositePass.java
index 7c5cd0b..2acedb4 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/TransientResourceCompositePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/PadoCompositePass.java
@@ -15,25 +15,24 @@
*/
package edu.snu.nemo.compiler.optimizer.pass.compiletime.composite;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.TransientResourceDataFlowPass;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.TransientResourceDataStorePass;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.TransientResourcePriorityPass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.PadoEdgeDataFlowModelPass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.PadoEdgeDataStorePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.PadoVertexExecutorPlacementPass;
import java.util.Arrays;
/**
- * A series of passes to harness transient resources.
- * Ref: https://dl.acm.org/citation.cfm?id=3064181
+ * A series of passes to support Pado optimization.
*/
-public final class TransientResourceCompositePass extends CompositePass {
+public final class PadoCompositePass extends CompositePass {
/**
* Default constructor.
*/
- public TransientResourceCompositePass() {
+ public PadoCompositePass() {
super(Arrays.asList(
- new TransientResourcePriorityPass(),
- new TransientResourceDataStorePass(),
- new TransientResourceDataFlowPass()
+ new PadoVertexExecutorPlacementPass(),
+ new PadoEdgeDataStorePass(),
+ new PadoEdgeDataFlowModelPass()
));
}
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DefaultCompositePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/PrimitiveCompositePass.java
similarity index 79%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DefaultCompositePass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/PrimitiveCompositePass.java
index 4dc16d4..dc8abfc 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DefaultCompositePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/PrimitiveCompositePass.java
@@ -24,23 +24,23 @@ import java.util.Arrays;
* It is highly recommended to place reshaping passes before this pass,
* and annotating passes after that and before this pass.
*/
-public final class DefaultCompositePass extends CompositePass {
+public final class PrimitiveCompositePass extends CompositePass {
/**
* Default constructor.
*/
- public DefaultCompositePass() {
+ public PrimitiveCompositePass() {
super(Arrays.asList(
new DefaultParallelismPass(),
new DefaultEdgeEncoderPass(),
new DefaultEdgeDecoderPass(),
- new DefaultDataStorePass(),
- new DefaultDataPersistencePass(),
+ new DefaultInterTaskDataStorePass(),
+ new DefaultEdgeUsedDataHandlingPass(),
new DefaultScheduleGroupPass(),
new CompressionPass(),
new DecompressionPass(),
- new ResourceLocalityPass(),
- new ResourceSitePass(),
- new ResourceSlotPass()
+ new SourceLocationAwareSchedulingPass(),
+ new NodeNamesAssignmentPass(),
+ new ExecutorSlotCompliancePass()
));
}
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/SailfishPass.java
similarity index 61%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/SailfishPass.java
index c6108e2..8ffd41f 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/SailfishPass.java
@@ -16,27 +16,26 @@
package edu.snu.nemo.compiler.optimizer.pass.compiletime.composite;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.*;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.reshaping.LargeShuffleRelayReshapingPass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.reshaping.SailfishRelayReshapingPass;
import java.util.Arrays;
/**
- * A series of passes to optimize large shuffle with disk seek batching techniques.
- * Ref. https://dl.acm.org/citation.cfm?id=2391233
+ * A series of passes to support Sailfish-like disk seek batching during shuffle.
*/
-public final class LargeShuffleCompositePass extends CompositePass {
+public final class SailfishPass extends CompositePass {
/**
* Default constructor.
*/
- public LargeShuffleCompositePass() {
+ public SailfishPass() {
super(Arrays.asList(
- new LargeShuffleRelayReshapingPass(),
- new LargeShuffleDataFlowPass(),
- new LargeShuffleDataStorePass(),
- new LargeShuffleDecoderPass(),
- new LargeShuffleEncoderPass(),
- new LargeShuffleDataPersistencePass(),
- new LargeShuffleResourceSlotPass()
+ new SailfishRelayReshapingPass(),
+ new SailfishEdgeDataFlowModelPass(),
+ new SailfishEdgeDataStorePass(),
+ new SailfishEdgeDecoderPass(),
+ new SailfishEdgeEncoderPass(),
+ new SailfishEdgeUsedDataHandlingPass(),
+ new SailfishVertexExecutorSlotCompliancePass()
));
}
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPass.java
index 606d84b..eef0b20 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPass.java
@@ -18,9 +18,9 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.reshaping;
import edu.snu.nemo.common.coder.DecoderFactory;
import edu.snu.nemo.common.coder.EncoderFactory;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DecoderProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.EncoderProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
import edu.snu.nemo.common.ir.vertex.transform.Transform;
@@ -41,7 +41,7 @@ public final class CommonSubexpressionEliminationPass extends ReshapingPass {
* Default constructor.
*/
public CommonSubexpressionEliminationPass() {
- super(Collections.singleton(CommunicationPatternProperty.class));
+ super(Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -150,7 +150,7 @@ public final class CommonSubexpressionEliminationPass extends ReshapingPass {
final Set<IREdge> outListToModify = outEdges.get(ov);
outEdges.getOrDefault(ov, new HashSet<>()).forEach(e -> {
outListToModify.remove(e);
- final IREdge newIrEdge = new IREdge(e.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge newIrEdge = new IREdge(e.getPropertyValue(DataCommunicationPatternProperty.class).get(),
operatorVertexToUse, e.getDst());
final Optional<EncoderFactory> encoderProperty = e.getPropertyValue(EncoderProperty.class);
if (encoderProperty.isPresent()) {
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/SkewReshapingPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/DataSkewReshapingPass.java
similarity index 81%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/SkewReshapingPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/DataSkewReshapingPass.java
index f7074bb..1117a7c 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/SkewReshapingPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/DataSkewReshapingPass.java
@@ -18,9 +18,9 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.reshaping;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DecoderProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.EncoderProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.MetricCollectionBarrierVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
@@ -33,16 +33,16 @@ import java.util.List;
* Pass to modify the DAG for a job to perform data skew.
* It adds a {@link MetricCollectionBarrierVertex} before Shuffle edges, to make a barrier before it,
* and to use the metrics to repartition the skewed data.
- * NOTE: we currently put the SkewCompositePass at the end of the list for each policies, as it needs to take
+ * NOTE: we currently put the DataSkewCompositePass at the end of the list for each policies, as it needs to take
* a snapshot at the end of the pass. This could be prevented by modifying other passes to take the snapshot of the
* DAG at the end of each passes for metricCollectionVertices.
*/
-public final class SkewReshapingPass extends ReshapingPass {
+public final class DataSkewReshapingPass extends ReshapingPass {
/**
* Default constructor.
*/
- public SkewReshapingPass() {
- super(Collections.singleton(CommunicationPatternProperty.class));
+ public DataSkewReshapingPass() {
+ super(Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -53,8 +53,8 @@ public final class SkewReshapingPass extends ReshapingPass {
dag.topologicalDo(v -> {
// We care about OperatorVertices that have any incoming edges that are of type Shuffle.
if (v instanceof OperatorVertex && dag.getIncomingEdgesOf(v).stream().anyMatch(irEdge ->
- CommunicationPatternProperty.Value.Shuffle
- .equals(irEdge.getPropertyValue(CommunicationPatternProperty.class).get()))) {
+ DataCommunicationPatternProperty.Value.Shuffle
+ .equals(irEdge.getPropertyValue(DataCommunicationPatternProperty.class).get()))) {
final MetricCollectionBarrierVertex<Integer, Long> metricCollectionBarrierVertex
= new MetricCollectionBarrierVertex<>();
metricCollectionVertices.add(metricCollectionBarrierVertex);
@@ -62,15 +62,15 @@ public final class SkewReshapingPass extends ReshapingPass {
builder.addVertex(metricCollectionBarrierVertex);
dag.getIncomingEdgesOf(v).forEach(edge -> {
// we insert the metric collection vertex when we meet a shuffle edge
- if (CommunicationPatternProperty.Value.Shuffle
- .equals(edge.getPropertyValue(CommunicationPatternProperty.class).get())) {
+ if (DataCommunicationPatternProperty.Value.Shuffle
+ .equals(edge.getPropertyValue(DataCommunicationPatternProperty.class).get())) {
// We then insert the dynamicOptimizationVertex between the vertex and incoming vertices.
- final IREdge newEdge = new IREdge(CommunicationPatternProperty.Value.OneToOne,
+ final IREdge newEdge = new IREdge(DataCommunicationPatternProperty.Value.OneToOne,
edge.getSrc(), metricCollectionBarrierVertex);
newEdge.setProperty(EncoderProperty.of(edge.getPropertyValue(EncoderProperty.class).get()));
newEdge.setProperty(DecoderProperty.of(edge.getPropertyValue(DecoderProperty.class).get()));
- final IREdge edgeToGbK = new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge edgeToGbK = new IREdge(edge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
metricCollectionBarrierVertex, v, edge.isSideInput());
edge.copyExecutionPropertiesTo(edgeToGbK);
builder.connectVertices(newEdge);
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPass.java
index 6c1afb2..c686eca 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopExtractionPass.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.compiler.optimizer.pass.compiletime.reshaping;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
@@ -37,7 +37,7 @@ public final class LoopExtractionPass extends ReshapingPass {
* Default constructor.
*/
public LoopExtractionPass() {
- super(Collections.singleton(CommunicationPatternProperty.class));
+ super(Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -99,7 +99,7 @@ public final class LoopExtractionPass extends ReshapingPass {
final LoopVertex srcLoopVertex = dag.getAssignedLoopVertexOf(irEdge.getSrc());
srcLoopVertex.addDagOutgoingEdge(irEdge);
final IREdge edgeFromLoop =
- new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ new IREdge(irEdge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
srcLoopVertex, operatorVertex, irEdge.isSideInput());
irEdge.copyExecutionPropertiesTo(edgeFromLoop);
builder.connectVertices(edgeFromLoop);
@@ -147,7 +147,7 @@ public final class LoopExtractionPass extends ReshapingPass {
assignedLoopVertex.getBuilder().connectVertices(irEdge);
} else { // loop -> loop connection
assignedLoopVertex.addDagIncomingEdge(irEdge);
- final IREdge edgeToLoop = new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge edgeToLoop = new IREdge(irEdge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
srcLoopVertex, assignedLoopVertex, irEdge.isSideInput());
irEdge.copyExecutionPropertiesTo(edgeToLoop);
builder.connectVertices(edgeToLoop);
@@ -155,7 +155,7 @@ public final class LoopExtractionPass extends ReshapingPass {
}
} else { // operator -> loop
assignedLoopVertex.addDagIncomingEdge(irEdge);
- final IREdge edgeToLoop = new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge edgeToLoop = new IREdge(irEdge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
irEdge.getSrc(), assignedLoopVertex, irEdge.isSideInput());
irEdge.copyExecutionPropertiesTo(edgeToLoop);
builder.connectVertices(edgeToLoop);
@@ -226,13 +226,13 @@ public final class LoopExtractionPass extends ReshapingPass {
final IRVertex equivalentSrcVertex = equivalentVertices.get(srcVertex);
// add the new IREdge to the iterative incoming edges list.
- final IREdge newIrEdge = new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge newIrEdge = new IREdge(edge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
equivalentSrcVertex, equivalentDstVertex, edge.isSideInput());
edge.copyExecutionPropertiesTo(newIrEdge);
finalRootLoopVertex.addIterativeIncomingEdge(newIrEdge);
} else {
// src is from outside the previous loop. vertex outside previous loop -> DAG.
- final IREdge newIrEdge = new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge newIrEdge = new IREdge(edge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
srcVertex, equivalentDstVertex, edge.isSideInput());
edge.copyExecutionPropertiesTo(newIrEdge);
finalRootLoopVertex.addNonIterativeIncomingEdge(newIrEdge);
@@ -245,7 +245,7 @@ public final class LoopExtractionPass extends ReshapingPass {
final IRVertex dstVertex = edge.getDst();
final IRVertex equivalentSrcVertex = equivalentVertices.get(srcVertex);
- final IREdge newIrEdge = new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge newIrEdge = new IREdge(edge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
equivalentSrcVertex, dstVertex, edge.isSideInput());
edge.copyExecutionPropertiesTo(newIrEdge);
finalRootLoopVertex.addDagOutgoingEdge(newIrEdge);
@@ -290,7 +290,7 @@ public final class LoopExtractionPass extends ReshapingPass {
if (edge.getSrc().equals(firstEquivalentVertex)) {
builder.connectVertices(edge);
} else {
- final IREdge newIrEdge = new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge newIrEdge = new IREdge(edge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
firstEquivalentVertex, irVertex, edge.isSideInput());
edge.copyExecutionPropertiesTo(newIrEdge);
builder.connectVertices(newIrEdge);
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopOptimizations.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopOptimizations.java
index 4dc53ac..072c163 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopOptimizations.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopOptimizations.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.compiler.optimizer.pass.compiletime.reshaping;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DecoderProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.EncoderProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
@@ -103,7 +103,7 @@ public final class LoopOptimizations {
* Default constructor.
*/
public LoopFusionPass() {
- super(Collections.singleton(CommunicationPatternProperty.class));
+ super(Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -160,7 +160,7 @@ public final class LoopOptimizations {
// inEdges.
inEdges.getOrDefault(loopVertex, new ArrayList<>()).forEach(irEdge -> {
if (builder.contains(irEdge.getSrc())) {
- final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class)
+ final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(DataCommunicationPatternProperty.class)
.get(), irEdge.getSrc(), newLoopVertex, irEdge.isSideInput());
irEdge.copyExecutionPropertiesTo(newIREdge);
builder.connectVertices(newIREdge);
@@ -169,7 +169,7 @@ public final class LoopOptimizations {
// outEdges.
outEdges.getOrDefault(loopVertex, new ArrayList<>()).forEach(irEdge -> {
if (builder.contains(irEdge.getDst())) {
- final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class)
+ final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(DataCommunicationPatternProperty.class)
.get(), newLoopVertex, irEdge.getDst(), irEdge.isSideInput());
irEdge.copyExecutionPropertiesTo(newIREdge);
builder.connectVertices(newIREdge);
@@ -251,7 +251,7 @@ public final class LoopOptimizations {
* Default constructor.
*/
public LoopInvariantCodeMotionPass() {
- super(Collections.singleton(CommunicationPatternProperty.class));
+ super(Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -287,7 +287,7 @@ public final class LoopOptimizations {
candidate.getValue().stream().map(IREdge::getSrc).anyMatch(edgeSrc -> edgeSrc.equals(e.getSrc())))
.forEach(edge -> {
edgesToRemove.add(edge);
- final IREdge newEdge = new IREdge(edge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge newEdge = new IREdge(edge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
candidate.getKey(), edge.getDst(), edge.isSideInput());
newEdge.setProperty(EncoderProperty.of(edge.getPropertyValue(EncoderProperty.class).get()));
newEdge.setProperty(DecoderProperty.of(edge.getPropertyValue(DecoderProperty.class).get()));
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LargeShuffleRelayReshapingPass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/SailfishRelayReshapingPass.java
similarity index 79%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LargeShuffleRelayReshapingPass.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/SailfishRelayReshapingPass.java
index dcbece7..1abe092 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LargeShuffleRelayReshapingPass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/SailfishRelayReshapingPass.java
@@ -19,7 +19,7 @@ import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.executionproperty.DecoderProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.EncoderProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.IREdge;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
@@ -34,13 +34,13 @@ import java.util.Collections;
* receiving shuffle edges,
* to merge the shuffled data in memory and write to the disk at once.
*/
-public final class LargeShuffleRelayReshapingPass extends ReshapingPass {
+public final class SailfishRelayReshapingPass extends ReshapingPass {
/**
* Default constructor.
*/
- public LargeShuffleRelayReshapingPass() {
- super(Collections.singleton(CommunicationPatternProperty.class));
+ public SailfishRelayReshapingPass() {
+ super(Collections.singleton(DataCommunicationPatternProperty.class));
}
@Override
@@ -51,20 +51,20 @@ public final class LargeShuffleRelayReshapingPass extends ReshapingPass {
// We care about OperatorVertices that have any incoming edge that
// has Shuffle as data communication pattern.
if (v instanceof OperatorVertex && dag.getIncomingEdgesOf(v).stream().anyMatch(irEdge ->
- CommunicationPatternProperty.Value.Shuffle
- .equals(irEdge.getPropertyValue(CommunicationPatternProperty.class).get()))) {
+ DataCommunicationPatternProperty.Value.Shuffle
+ .equals(irEdge.getPropertyValue(DataCommunicationPatternProperty.class).get()))) {
dag.getIncomingEdgesOf(v).forEach(edge -> {
- if (CommunicationPatternProperty.Value.Shuffle
- .equals(edge.getPropertyValue(CommunicationPatternProperty.class).get())) {
+ if (DataCommunicationPatternProperty.Value.Shuffle
+ .equals(edge.getPropertyValue(DataCommunicationPatternProperty.class).get())) {
// Insert a merger vertex having transform that write received data immediately
// before the vertex receiving shuffled data.
final OperatorVertex iFileMergerVertex = new OperatorVertex(new RelayTransform());
iFileMergerVertex.getExecutionProperties().put(SkipSerDesProperty.of());
builder.addVertex(iFileMergerVertex);
- final IREdge newEdgeToMerger = new IREdge(CommunicationPatternProperty.Value.Shuffle,
+ final IREdge newEdgeToMerger = new IREdge(DataCommunicationPatternProperty.Value.Shuffle,
edge.getSrc(), iFileMergerVertex, edge.isSideInput());
edge.copyExecutionPropertiesTo(newEdgeToMerger);
- final IREdge newEdgeFromMerger = new IREdge(CommunicationPatternProperty.Value.OneToOne,
+ final IREdge newEdgeFromMerger = new IREdge(DataCommunicationPatternProperty.Value.OneToOne,
iFileMergerVertex, v);
newEdgeFromMerger.setProperty(EncoderProperty.of(edge.getPropertyValue(EncoderProperty.class).get()));
newEdgeFromMerger.setProperty(DecoderProperty.of(edge.getPropertyValue(DecoderProperty.class).get()));
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DataSkewPolicy.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DataSkewPolicy.java
index 03be205..4226bc6 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DataSkewPolicy.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DataSkewPolicy.java
@@ -16,9 +16,9 @@
package edu.snu.nemo.compiler.optimizer.policy;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.SkewCompositePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.DataSkewCompositePass;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.LoopOptimizationCompositePass;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.PrimitiveCompositePass;
import edu.snu.nemo.runtime.common.optimizer.pass.runtime.DataSkewRuntimePass;
import edu.snu.nemo.runtime.common.optimizer.pass.runtime.RuntimePass;
@@ -35,17 +35,17 @@ public final class DataSkewPolicy implements Policy {
*/
public DataSkewPolicy() {
this.policy = new PolicyBuilder(true)
- .registerRuntimePass(new DataSkewRuntimePass(), new SkewCompositePass())
+ .registerRuntimePass(new DataSkewRuntimePass(), new DataSkewCompositePass())
.registerCompileTimePass(new LoopOptimizationCompositePass())
- .registerCompileTimePass(new DefaultCompositePass())
+ .registerCompileTimePass(new PrimitiveCompositePass())
.build();
}
public DataSkewPolicy(final int skewness) {
this.policy = new PolicyBuilder(true)
- .registerRuntimePass(new DataSkewRuntimePass().setNumSkewedKeys(skewness), new SkewCompositePass())
+ .registerRuntimePass(new DataSkewRuntimePass().setNumSkewedKeys(skewness), new DataSkewCompositePass())
.registerCompileTimePass(new LoopOptimizationCompositePass())
- .registerCompileTimePass(new DefaultCompositePass())
+ .registerCompileTimePass(new PrimitiveCompositePass())
.build();
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DefaultPolicy.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DefaultPolicy.java
index 851e5dd..f02b6ef 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DefaultPolicy.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DefaultPolicy.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.compiler.optimizer.policy;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.PrimitiveCompositePass;
import edu.snu.nemo.runtime.common.optimizer.pass.runtime.RuntimePass;
import java.util.List;
@@ -32,7 +32,7 @@ public final class DefaultPolicy implements Policy {
*/
public DefaultPolicy() {
this.policy = new PolicyBuilder(true)
- .registerCompileTimePass(new DefaultCompositePass())
+ .registerCompileTimePass(new PrimitiveCompositePass())
.build();
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DefaultPolicyWithSeparatePass.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DefaultPolicyWithSeparatePass.java
index b139ee2..d23d5bd 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DefaultPolicyWithSeparatePass.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DefaultPolicyWithSeparatePass.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.compiler.optimizer.policy;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultParallelismPass;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultDataStorePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultInterTaskDataStorePass;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultScheduleGroupPass;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.CompositePass;
@@ -62,7 +62,7 @@ public final class DefaultPolicyWithSeparatePass implements Policy {
*/
RefactoredPass() {
super(Arrays.asList(
- new DefaultDataStorePass(),
+ new DefaultInterTaskDataStorePass(),
new DefaultScheduleGroupPass()
));
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DisaggregationPolicy.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DisaggregationPolicy.java
index 8d69b24..4f27ef3 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DisaggregationPolicy.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/DisaggregationPolicy.java
@@ -17,7 +17,7 @@ package edu.snu.nemo.compiler.optimizer.policy;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.*;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.*;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.PrimitiveCompositePass;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.LoopOptimizationCompositePass;
import edu.snu.nemo.runtime.common.optimizer.pass.runtime.RuntimePass;
@@ -35,7 +35,7 @@ public final class DisaggregationPolicy implements Policy {
public DisaggregationPolicy() {
this.policy = new PolicyBuilder(false)
.registerCompileTimePass(new LoopOptimizationCompositePass())
- .registerCompileTimePass(new DefaultCompositePass())
+ .registerCompileTimePass(new PrimitiveCompositePass())
.registerCompileTimePass(new DisaggregationEdgeDataStorePass())
.build();
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/TransientResourcePolicy.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/PadoPolicy.java
similarity index 79%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/TransientResourcePolicy.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/PadoPolicy.java
index 19baade..e2855fc 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/TransientResourcePolicy.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/PadoPolicy.java
@@ -16,27 +16,28 @@
package edu.snu.nemo.compiler.optimizer.policy;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.*;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.PrimitiveCompositePass;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.LoopOptimizationCompositePass;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.TransientResourceCompositePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.PadoCompositePass;
import edu.snu.nemo.runtime.common.optimizer.pass.runtime.RuntimePass;
import java.util.List;
/**
- * A policy to perform optimization that uses transient resources in data centers.
+ * A policy to perform Pado optimization that uses transient resources on data centers.
+ * link to paper: http://dl.acm.org/citation.cfm?id=3064181
*/
-public final class TransientResourcePolicy implements Policy {
+public final class PadoPolicy implements Policy {
private final Policy policy;
/**
* Default constructor.
*/
- public TransientResourcePolicy() {
+ public PadoPolicy() {
this.policy = new PolicyBuilder(true)
- .registerCompileTimePass(new TransientResourceCompositePass())
+ .registerCompileTimePass(new PadoCompositePass())
.registerCompileTimePass(new LoopOptimizationCompositePass())
- .registerCompileTimePass(new DefaultCompositePass())
+ .registerCompileTimePass(new PrimitiveCompositePass())
.build();
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/PolicyBuilder.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/PolicyBuilder.java
index 9928991..5268c00 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/PolicyBuilder.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/PolicyBuilder.java
@@ -16,12 +16,12 @@
package edu.snu.nemo.compiler.optimizer.policy;
import edu.snu.nemo.common.exception.CompileTimeOptimizationException;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.PartitionerProperty;
import edu.snu.nemo.common.ir.executionproperty.ExecutionProperty;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
import edu.snu.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.AnnotatingPass;
@@ -62,12 +62,12 @@ public final class PolicyBuilder {
this.annotatedExecutionProperties = new HashSet<>();
this.strictPrerequisiteCheckMode = strictPrerequisiteCheckMode;
// DataCommunicationPattern is already set when creating the IREdge itself.
- annotatedExecutionProperties.add(CommunicationPatternProperty.class);
+ annotatedExecutionProperties.add(DataCommunicationPatternProperty.class);
// Some default values are already annotated.
- annotatedExecutionProperties.add(ResourcePriorityProperty.class);
+ annotatedExecutionProperties.add(ExecutorPlacementProperty.class);
annotatedExecutionProperties.add(ParallelismProperty.class);
- annotatedExecutionProperties.add(DataFlowProperty.class);
- annotatedExecutionProperties.add(DataStoreProperty.class);
+ annotatedExecutionProperties.add(DataFlowModelProperty.class);
+ annotatedExecutionProperties.add(InterTaskDataStoreProperty.class);
annotatedExecutionProperties.add(PartitionerProperty.class);
}
diff --git a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/LargeShufflePolicy.java b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/SailfishPolicy.java
similarity index 80%
rename from compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/LargeShufflePolicy.java
rename to compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/SailfishPolicy.java
index 7f01151..c53b228 100644
--- a/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/LargeShufflePolicy.java
+++ b/compiler/optimizer/src/main/java/edu/snu/nemo/compiler/optimizer/policy/SailfishPolicy.java
@@ -16,27 +16,27 @@
package edu.snu.nemo.compiler.optimizer.policy;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.DefaultCompositePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.PrimitiveCompositePass;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.LoopOptimizationCompositePass;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.LargeShuffleCompositePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.SailfishPass;
import edu.snu.nemo.runtime.common.optimizer.pass.runtime.RuntimePass;
import java.util.List;
/**
- * A policy to demonstrate the large shuffle optimization, witch batches disk seek during data shuffle.
+ * A policy to demonstrate the Sailfish optimization, that batches disk seek during data shuffle.
*/
-public final class LargeShufflePolicy implements Policy {
+public final class SailfishPolicy implements Policy {
private final Policy policy;
/**
* Default constructor.
*/
- public LargeShufflePolicy() {
+ public SailfishPolicy() {
this.policy = new PolicyBuilder(false)
- .registerCompileTimePass(new LargeShuffleCompositePass())
+ .registerCompileTimePass(new SailfishPass())
.registerCompileTimePass(new LoopOptimizationCompositePass())
- .registerCompileTimePass(new DefaultCompositePass())
+ .registerCompileTimePass(new PrimitiveCompositePass())
.build();
}
diff --git a/compiler/optimizer/src/test/java/edu/snu/nemo/compiler/optimizer/policy/PolicyBuilderTest.java b/compiler/optimizer/src/test/java/edu/snu/nemo/compiler/optimizer/policy/PolicyBuilderTest.java
index 4503fbe..7f96cfa 100644
--- a/compiler/optimizer/src/test/java/edu/snu/nemo/compiler/optimizer/policy/PolicyBuilderTest.java
+++ b/compiler/optimizer/src/test/java/edu/snu/nemo/compiler/optimizer/policy/PolicyBuilderTest.java
@@ -17,7 +17,7 @@ package edu.snu.nemo.compiler.optimizer.policy;
import edu.snu.nemo.common.exception.CompileTimeOptimizationException;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultScheduleGroupPass;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.TransientResourceCompositePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.composite.PadoCompositePass;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
@@ -32,10 +32,10 @@ public final class PolicyBuilderTest {
}
@Test
- public void testTransientResourcePolicy() {
- final Policy transientResourcePolicy = new TransientResourcePolicy();
- assertEquals(19, transientResourcePolicy.getCompileTimePasses().size());
- assertEquals(0, transientResourcePolicy.getRuntimePasses().size());
+ public void testPadoPolicy() {
+ final Policy padoPolicy = new PadoPolicy();
+ assertEquals(19, padoPolicy.getCompileTimePasses().size());
+ assertEquals(0, padoPolicy.getRuntimePasses().size());
}
@Test
@@ -49,7 +49,7 @@ public final class PolicyBuilderTest {
public void testShouldFailPolicy() {
try {
final Policy failPolicy = new PolicyBuilder()
- .registerCompileTimePass(new TransientResourceCompositePass())
+ .registerCompileTimePass(new PadoCompositePass())
.registerCompileTimePass(new DefaultScheduleGroupPass())
.build();
} catch (Exception e) { // throw an exception if default execution properties are not set.
diff --git a/compiler/test/src/test/java/edu/snu/nemo/compiler/backend/nemo/DAGConverterTest.java b/compiler/test/src/test/java/edu/snu/nemo/compiler/backend/nemo/DAGConverterTest.java
index 79721fe..6702b44 100644
--- a/compiler/test/src/test/java/edu/snu/nemo/compiler/backend/nemo/DAGConverterTest.java
+++ b/compiler/test/src/test/java/edu/snu/nemo/compiler/backend/nemo/DAGConverterTest.java
@@ -18,13 +18,13 @@ package edu.snu.nemo.compiler.backend.nemo;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
import edu.snu.nemo.common.ir.vertex.SourceVertex;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
import edu.snu.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
import edu.snu.nemo.common.ir.vertex.transform.Transform;
import edu.snu.nemo.compiler.frontend.beam.transform.DoTransform;
@@ -71,7 +71,7 @@ public final class DAGConverterTest {
v2.setProperty(ParallelismProperty.of(2));
irDAGBuilder.addVertex(v2);
- final IREdge e = new IREdge(CommunicationPatternProperty.Value.Shuffle, v1, v2);
+ final IREdge e = new IREdge(DataCommunicationPatternProperty.Value.Shuffle, v1, v2);
irDAGBuilder.connectVertices(e);
final DAG<IRVertex, IREdge> irDAG = CompiletimeOptimizer.optimize(irDAGBuilder.buildWithoutSourceSinkCheck(),
@@ -111,37 +111,37 @@ public final class DAGConverterTest {
final IRVertex v1 = s;
v1.setProperty(ParallelismProperty.of(3));
- v1.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v1.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
final Transform t = mock(Transform.class);
final DoTransform dt = new DoTransform(null, null);
final IRVertex v2 = new OperatorVertex(t);
v2.setProperty(ParallelismProperty.of(3));
- v2.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v2.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
final IRVertex v3 = new OperatorVertex(t);
v3.setProperty(ParallelismProperty.of(3));
- v3.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v3.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
final IRVertex v4 = new OperatorVertex(t);
v4.setProperty(ParallelismProperty.of(2));
- v4.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v4.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
final IRVertex v5 = new OperatorVertex(dt);
v5.setProperty(ParallelismProperty.of(2));
- v5.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v5.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
final IRVertex v6 = new OperatorVertex(dt);
v6.setProperty(ParallelismProperty.of(2));
- v6.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.RESERVED));
+ v6.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.RESERVED));
// final IRVertex v7 = new OperatorVertex(t);
// v7.setProperty(Parallelism.of(2));
-// v7.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+// v7.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
final IRVertex v8 = new OperatorVertex(dt);
v8.setProperty(ParallelismProperty.of(2));
- v8.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v8.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
irDAGBuilder.addVertex(v1);
irDAGBuilder.addVertex(v2);
@@ -153,37 +153,37 @@ public final class DAGConverterTest {
// irDAGBuilder.addVertex(v7);
- final IREdge e1 = new IREdge(CommunicationPatternProperty.Value.OneToOne, v1, v2);
- e1.setProperty(DataStoreProperty.of(DataStoreProperty.Value.MemoryStore));
- e1.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Pull));
+ final IREdge e1 = new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v1, v2);
+ e1.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.MemoryStore));
+ e1.setProperty(DataFlowModelProperty.of(DataFlowModelProperty.Value.Pull));
- final IREdge e2 = new IREdge(CommunicationPatternProperty.Value.OneToOne, v1, v3);
- e2.setProperty(DataStoreProperty.of(DataStoreProperty.Value.MemoryStore));
- e2.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Pull));
+ final IREdge e2 = new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v1, v3);
+ e2.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.MemoryStore));
+ e2.setProperty(DataFlowModelProperty.of(DataFlowModelProperty.Value.Pull));
- final IREdge e3 = new IREdge(CommunicationPatternProperty.Value.Shuffle, v2, v4);
- e3.setProperty(DataStoreProperty.of(DataStoreProperty.Value.MemoryStore));
- e3.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Push));
+ final IREdge e3 = new IREdge(DataCommunicationPatternProperty.Value.Shuffle, v2, v4);
+ e3.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.MemoryStore));
+ e3.setProperty(DataFlowModelProperty.of(DataFlowModelProperty.Value.Push));
- final IREdge e4 = new IREdge(CommunicationPatternProperty.Value.Shuffle, v3, v5);
- e4.setProperty(DataStoreProperty.of(DataStoreProperty.Value.MemoryStore));
- e4.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Push));
+ final IREdge e4 = new IREdge(DataCommunicationPatternProperty.Value.Shuffle, v3, v5);
+ e4.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.MemoryStore));
+ e4.setProperty(DataFlowModelProperty.of(DataFlowModelProperty.Value.Push));
- final IREdge e5 = new IREdge(CommunicationPatternProperty.Value.OneToOne, v4, v6);
- e5.setProperty(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore));
- e5.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Pull));
+ final IREdge e5 = new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v4, v6);
+ e5.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.LocalFileStore));
+ e5.setProperty(DataFlowModelProperty.of(DataFlowModelProperty.Value.Pull));
- final IREdge e6 = new IREdge(CommunicationPatternProperty.Value.OneToOne, v4, v8);
- e6.setProperty(DataStoreProperty.of(DataStoreProperty.Value.LocalFileStore));
- e6.setProperty(DataFlowProperty.of(DataFlowProperty.Value.Pull));
+ final IREdge e6 = new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v4, v8);
+ e6.setProperty(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.LocalFileStore));
+ e6.setProperty(DataFlowModelProperty.of(DataFlowModelProperty.Value.Pull));
// final IREdge e7 = new IREdge(OneToOne, v7, v5);
-// e7.setProperty(DataStoreProperty.of(MemoryStore));
-// e7.setProperty(Attribute.Key.PullOrPush, DataFlowProperty.Value.Push));
+// e7.setProperty(InterTaskDataStoreProperty.of(MemoryStore));
+// e7.setProperty(Attribute.Key.PullOrPush, DataFlowModelProperty.Value.Push));
//
// final IREdge e8 = new IREdge(OneToOne, v5, v8);
-// e8.setProperty(DataStoreProperty.of(MemoryStore));
-// e8.setProperty(Attribute.Key.PullOrPush, DataFlowProperty.Value.Pull));
+// e8.setProperty(InterTaskDataStoreProperty.of(MemoryStore));
+// e8.setProperty(Attribute.Key.PullOrPush, DataFlowModelProperty.Value.Pull));
// Stage 1 = {v1, v2, v3}
irDAGBuilder.connectVertices(e1);
diff --git a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPassTest.java b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPassTest.java
index 64d0bce..585c40b 100644
--- a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPassTest.java
+++ b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/annotating/DefaultScheduleGroupPassTest.java
@@ -20,11 +20,13 @@ import edu.snu.nemo.common.Pair;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
import edu.snu.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty;
+import edu.snu.nemo.common.ir.vertex.transform.Transform;
+import edu.snu.nemo.common.test.EmptyComponents;
import edu.snu.nemo.compiler.CompilerTestUtil;
import edu.snu.nemo.compiler.optimizer.CompiletimeOptimizer;
import edu.snu.nemo.compiler.optimizer.policy.TestPolicy;
@@ -77,13 +79,13 @@ public final class DefaultScheduleGroupPassTest {
* v0 --- v1 --- v2 --/
* }
*
- * @param communicationPattern {@link CommunicationPatternProperty.Value} for the edges
- * @param dataFlowModel {@link DataFlowProperty.Value} for the edges
+ * @param communicationPattern {@link DataCommunicationPatternProperty.Value} for the edges
+ * @param dataFlowModel {@link DataFlowModelProperty.Value} for the edges
* @return a {@link Pair} of {@link DAG} and {@link List} of {@link IRVertex}
*/
private static Pair<DAG<IRVertex, IREdge>, List<IRVertex>> generateBranchDAG(
- final CommunicationPatternProperty.Value communicationPattern,
- final DataFlowProperty.Value dataFlowModel) {
+ final DataCommunicationPatternProperty.Value communicationPattern,
+ final DataFlowModelProperty.Value dataFlowModel) {
final DAGBuilder<IRVertex, IREdge> dagBuilder = new DAGBuilder<>();
final IRVertex v0 = new OperatorVertex(EMPTY_TRANSFORM);
@@ -103,7 +105,7 @@ public final class DefaultScheduleGroupPassTest {
dagBuilder.addVertex(vertex);
}
for (final IREdge edge : Arrays.asList(e0, e1, e2, e3, e4)) {
- edge.getExecutionProperties().put(DataFlowProperty.of(dataFlowModel));
+ edge.getExecutionProperties().put(DataFlowModelProperty.of(dataFlowModel));
dagBuilder.connectVertices(edge);
}
return Pair.of(dagBuilder.buildWithoutSourceSinkCheck(), vertices);
@@ -116,13 +118,13 @@ public final class DefaultScheduleGroupPassTest {
* v2 --- v3 --/
* }
*
- * @param communicationPattern {@link CommunicationPatternProperty.Value} for the edges
- * @param dataFlowModel {@link DataFlowProperty.Value} for the edges
+ * @param communicationPattern {@link DataCommunicationPatternProperty.Value} for the edges
+ * @param dataFlowModel {@link DataFlowModelProperty.Value} for the edges
* @return a {@link Pair} of {@link DAG} and {@link List} of {@link IRVertex}
*/
private static Pair<DAG<IRVertex, IREdge>, List<IRVertex>> generateJoinDAG(
- final CommunicationPatternProperty.Value communicationPattern,
- final DataFlowProperty.Value dataFlowModel) {
+ final DataCommunicationPatternProperty.Value communicationPattern,
+ final DataFlowModelProperty.Value dataFlowModel) {
final DAGBuilder<IRVertex, IREdge> dagBuilder = new DAGBuilder<>();
final IRVertex v0 = new OperatorVertex(EMPTY_TRANSFORM);
@@ -143,7 +145,7 @@ public final class DefaultScheduleGroupPassTest {
dagBuilder.addVertex(vertex);
}
for (final IREdge edge : Arrays.asList(e0, e1, e2, e3, e4)) {
- edge.getExecutionProperties().put(DataFlowProperty.of(dataFlowModel));
+ edge.getExecutionProperties().put(DataFlowModelProperty.of(dataFlowModel));
dagBuilder.connectVertices(edge);
}
return Pair.of(dagBuilder.buildWithoutSourceSinkCheck(), vertices);
@@ -187,7 +189,7 @@ public final class DefaultScheduleGroupPassTest {
public void testBranch() {
final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass();
final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
- = generateBranchDAG(CommunicationPatternProperty.Value.OneToOne, DataFlowProperty.Value.Pull);
+ = generateBranchDAG(DataCommunicationPatternProperty.Value.OneToOne, DataFlowModelProperty.Value.Pull);
pass.apply(dag.left());
dag.right().forEach(v -> assertScheduleGroup(0, v));
}
@@ -199,7 +201,7 @@ public final class DefaultScheduleGroupPassTest {
public void testBranchWhenMultipleInEdgeNotAllowed() {
final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass(false, false, false);
final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
- = generateBranchDAG(CommunicationPatternProperty.Value.OneToOne, DataFlowProperty.Value.Pull);
+ = generateBranchDAG(DataCommunicationPatternProperty.Value.OneToOne, DataFlowModelProperty.Value.Pull);
pass.apply(dag.left());
dag.right().subList(0, 4).forEach(v -> assertScheduleGroup(0, v));
dag.right().subList(4, 5).forEach(v -> assertScheduleGroup(1, v));
@@ -212,7 +214,7 @@ public final class DefaultScheduleGroupPassTest {
public void testBranchWithPush() {
final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass(false, false, false);
final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
- = generateBranchDAG(CommunicationPatternProperty.Value.Shuffle, DataFlowProperty.Value.Push);
+ = generateBranchDAG(DataCommunicationPatternProperty.Value.Shuffle, DataFlowModelProperty.Value.Push);
pass.apply(dag.left());
dag.right().forEach(v -> assertScheduleGroup(0, v));
}
@@ -224,7 +226,7 @@ public final class DefaultScheduleGroupPassTest {
public void testBranchWithBroadcast() {
final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass(false, true, true);
final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
- = generateBranchDAG(CommunicationPatternProperty.Value.BroadCast, DataFlowProperty.Value.Pull);
+ = generateBranchDAG(DataCommunicationPatternProperty.Value.BroadCast, DataFlowModelProperty.Value.Pull);
assertDifferentScheduleGroup(pass.apply(dag.left()).getVertices());
}
@@ -235,7 +237,7 @@ public final class DefaultScheduleGroupPassTest {
public void testBranchWithShuffle() {
final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass(true, false, true);
final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
- = generateBranchDAG(CommunicationPatternProperty.Value.Shuffle, DataFlowProperty.Value.Pull);
+ = generateBranchDAG(DataCommunicationPatternProperty.Value.Shuffle, DataFlowModelProperty.Value.Pull);
assertDifferentScheduleGroup(pass.apply(dag.left()).getVertices());
}
@@ -246,7 +248,7 @@ public final class DefaultScheduleGroupPassTest {
public void testJoin() {
final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass();
final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
- = generateJoinDAG(CommunicationPatternProperty.Value.OneToOne, DataFlowProperty.Value.Pull);
+ = generateJoinDAG(DataCommunicationPatternProperty.Value.OneToOne, DataFlowModelProperty.Value.Pull);
pass.apply(dag.left());
final int idxForFirstScheduleGroup = getScheduleGroup(dag.right().get(0));
final int idxForSecondScheduleGroup = getScheduleGroup(dag.right().get(2));
@@ -262,7 +264,7 @@ public final class DefaultScheduleGroupPassTest {
public void testJoinWithPush() {
final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass();
final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
- = generateJoinDAG(CommunicationPatternProperty.Value.OneToOne, DataFlowProperty.Value.Push);
+ = generateJoinDAG(DataCommunicationPatternProperty.Value.OneToOne, DataFlowModelProperty.Value.Push);
pass.apply(dag.left());
dag.right().forEach(v -> assertScheduleGroup(0, v));
}
@@ -274,9 +276,9 @@ public final class DefaultScheduleGroupPassTest {
public void testJoinWithSinglePush() {
final DefaultScheduleGroupPass pass = new DefaultScheduleGroupPass();
final Pair<DAG<IRVertex, IREdge>, List<IRVertex>> dag
- = generateJoinDAG(CommunicationPatternProperty.Value.OneToOne, DataFlowProperty.Value.Push);
+ = generateJoinDAG(DataCommunicationPatternProperty.Value.OneToOne, DataFlowModelProperty.Value.Push);
dag.left().getOutgoingEdgesOf(dag.right().get(1)).iterator().next()
- .getExecutionProperties().put(DataFlowProperty.of(DataFlowProperty.Value.Pull));
+ .getExecutionProperties().put(DataFlowModelProperty.of(DataFlowModelProperty.Value.Pull));
pass.apply(dag.left());
final int idxForFirstScheduleGroup = getScheduleGroup(dag.right().get(0));
final int idxForSecondScheduleGroup = getScheduleGroup(dag.right().get(2));
diff --git a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePassTest.java b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DataSkewCompositePassTest.java
similarity index 81%
rename from compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePassTest.java
rename to compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DataSkewCompositePassTest.java
index ed21561..4034748 100644
--- a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/SkewCompositePassTest.java
+++ b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DataSkewCompositePassTest.java
@@ -18,14 +18,14 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.composite;
import edu.snu.nemo.client.JobLauncher;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.MetricCollectionProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.PartitionerProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.MetricCollectionBarrierVertex;
import edu.snu.nemo.common.ir.executionproperty.ExecutionProperty;
import edu.snu.nemo.compiler.CompilerTestUtil;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceSkewedDataProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.SkewnessAwareSchedulingProperty;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.AnnotatingPass;
import org.junit.Before;
import org.junit.Test;
@@ -42,11 +42,11 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
/**
- * Test {@link SkewCompositePass} with MR workload.
+ * Test {@link DataSkewCompositePass} with MR workload.
*/
@RunWith(PowerMockRunner.class)
@PrepareForTest(JobLauncher.class)
-public class SkewCompositePassTest {
+public class DataSkewCompositePassTest {
private DAG<IRVertex, IREdge> mrDAG;
private static final long NUM_OF_PASSES_IN_DATA_SKEW_PASS = 5;
@@ -59,7 +59,7 @@ public class SkewCompositePassTest {
*/
@Test
public void testCompositePass() {
- final CompositePass dataSkewPass = new SkewCompositePass();
+ final CompositePass dataSkewPass = new DataSkewCompositePass();
assertEquals(NUM_OF_PASSES_IN_DATA_SKEW_PASS, dataSkewPass.getPassList().size());
final Set<Class<? extends ExecutionProperty>> prerequisites = new HashSet<>();
@@ -74,7 +74,7 @@ public class SkewCompositePassTest {
}
/**
- * Test for {@link SkewCompositePass} with MR workload. It must insert a {@link MetricCollectionBarrierVertex}
+ * Test for {@link DataSkewCompositePass} with MR workload. It must insert a {@link MetricCollectionBarrierVertex}
* before each shuffle edge.
* @throws Exception exception on the way.
*/
@@ -84,16 +84,16 @@ public class SkewCompositePassTest {
final Integer originalVerticesNum = mrDAG.getVertices().size();
final Long numOfShuffleGatherEdges = mrDAG.getVertices().stream().filter(irVertex ->
mrDAG.getIncomingEdgesOf(irVertex).stream().anyMatch(irEdge ->
- CommunicationPatternProperty.Value.Shuffle
- .equals(irEdge.getPropertyValue(CommunicationPatternProperty.class).get())))
+ DataCommunicationPatternProperty.Value.Shuffle
+ .equals(irEdge.getPropertyValue(DataCommunicationPatternProperty.class).get())))
.count();
- final DAG<IRVertex, IREdge> processedDAG = new SkewCompositePass().apply(mrDAG);
+ final DAG<IRVertex, IREdge> processedDAG = new DataSkewCompositePass().apply(mrDAG);
assertEquals(originalVerticesNum + numOfShuffleGatherEdges, processedDAG.getVertices().size());
processedDAG.getVertices().stream().map(processedDAG::getIncomingEdgesOf)
.flatMap(List::stream)
- .filter(irEdge -> CommunicationPatternProperty.Value.Shuffle
- .equals(irEdge.getPropertyValue(CommunicationPatternProperty.class).get()))
+ .filter(irEdge -> DataCommunicationPatternProperty.Value.Shuffle
+ .equals(irEdge.getPropertyValue(DataCommunicationPatternProperty.class).get()))
.map(IREdge::getSrc)
.forEach(irVertex -> assertTrue(irVertex instanceof MetricCollectionBarrierVertex));
@@ -106,7 +106,7 @@ public class SkewCompositePassTest {
processedDAG.filterVertices(v -> v instanceof MetricCollectionBarrierVertex)
.forEach(metricV -> {
List<IRVertex> reducerV = processedDAG.getChildren(metricV.getId());
- reducerV.forEach(rV -> assertTrue(rV.getPropertyValue(ResourceSkewedDataProperty.class).get()));
+ reducerV.forEach(rV -> assertTrue(rV.getPropertyValue(SkewnessAwareSchedulingProperty.class).get()));
});
}
}
diff --git a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DisaggregationPassTest.java b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DisaggregationPassTest.java
index d41bf36..98775d3 100644
--- a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DisaggregationPassTest.java
+++ b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/DisaggregationPassTest.java
@@ -18,12 +18,12 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.composite;
import edu.snu.nemo.client.JobLauncher;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.compiler.CompilerTestUtil;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultParallelismPass;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.DisaggregationEdgeDataStorePass;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultDataStorePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.DefaultInterTaskDataStorePass;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -49,12 +49,12 @@ public class DisaggregationPassTest {
public void testDisaggregation() throws Exception {
final DAG<IRVertex, IREdge> processedDAG =
new DisaggregationEdgeDataStorePass().apply(
- new DefaultDataStorePass().apply(
+ new DefaultInterTaskDataStorePass().apply(
new DefaultParallelismPass().apply(compiledDAG)));
processedDAG.getTopologicalSort().forEach(irVertex ->
processedDAG.getIncomingEdgesOf(irVertex).forEach(edgeToMerger ->
- assertEquals(DataStoreProperty.Value.GlusterFileStore,
- edgeToMerger.getPropertyValue(DataStoreProperty.class).get())));
+ assertEquals(InterTaskDataStoreProperty.Value.GlusterFileStore,
+ edgeToMerger.getPropertyValue(InterTaskDataStoreProperty.class).get())));
}
}
diff --git a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/PadoCompositePassTest.java b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/PadoCompositePassTest.java
new file mode 100644
index 0000000..3683088
--- /dev/null
+++ b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/PadoCompositePassTest.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright (C) 2018 Seoul National University
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.snu.nemo.compiler.optimizer.pass.compiletime.composite;
+
+import edu.snu.nemo.client.JobLauncher;
+import edu.snu.nemo.common.dag.DAG;
+import edu.snu.nemo.common.ir.edge.IREdge;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
+import edu.snu.nemo.common.ir.vertex.IRVertex;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
+import edu.snu.nemo.compiler.CompilerTestUtil;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.PadoEdgeDataStorePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.PadoVertexExecutorPlacementPass;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test {@link PadoVertexExecutorPlacementPass} and {@link PadoEdgeDataStorePass}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(JobLauncher.class)
+public class PadoCompositePassTest {
+ private DAG<IRVertex, IREdge> compiledDAG;
+
+ @Before
+ public void setUp() throws Exception {
+ compiledDAG = CompilerTestUtil.compileALSDAG();
+ }
+
+ @Test
+ public void testPadoPass() throws Exception {
+ final DAG<IRVertex, IREdge> processedDAG = new PadoCompositePass().apply(compiledDAG);
+
+ final IRVertex vertex1 = processedDAG.getTopologicalSort().get(0);
+ assertEquals(ExecutorPlacementProperty.TRANSIENT, vertex1.getPropertyValue(ExecutorPlacementProperty.class).get());
+
+ final IRVertex vertex5 = processedDAG.getTopologicalSort().get(1);
+ assertEquals(ExecutorPlacementProperty.TRANSIENT, vertex5.getPropertyValue(ExecutorPlacementProperty.class).get());
+ processedDAG.getIncomingEdgesOf(vertex5).forEach(irEdge -> {
+ assertEquals(InterTaskDataStoreProperty.Value.MemoryStore, irEdge.getPropertyValue(InterTaskDataStoreProperty.class).get());
+ assertEquals(DataFlowModelProperty.Value.Pull, irEdge.getPropertyValue(DataFlowModelProperty.class).get());
+ });
+
+ final IRVertex vertex6 = processedDAG.getTopologicalSort().get(2);
+ assertEquals(ExecutorPlacementProperty.RESERVED, vertex6.getPropertyValue(ExecutorPlacementProperty.class).get());
+ processedDAG.getIncomingEdgesOf(vertex6).forEach(irEdge -> {
+ assertEquals(InterTaskDataStoreProperty.Value.LocalFileStore, irEdge.getPropertyValue(InterTaskDataStoreProperty.class).get());
+ assertEquals(DataFlowModelProperty.Value.Push, irEdge.getPropertyValue(DataFlowModelProperty.class).get());
+ });
+
+ final IRVertex vertex4 = processedDAG.getTopologicalSort().get(6);
+ assertEquals(ExecutorPlacementProperty.RESERVED, vertex4.getPropertyValue(ExecutorPlacementProperty.class).get());
+ processedDAG.getIncomingEdgesOf(vertex4).forEach(irEdge -> {
+ assertEquals(InterTaskDataStoreProperty.Value.MemoryStore, irEdge.getPropertyValue(InterTaskDataStoreProperty.class).get());
+ assertEquals(DataFlowModelProperty.Value.Pull, irEdge.getPropertyValue(DataFlowModelProperty.class).get());
+ });
+
+ final IRVertex vertex12 = processedDAG.getTopologicalSort().get(10);
+ assertEquals(ExecutorPlacementProperty.TRANSIENT, vertex12.getPropertyValue(ExecutorPlacementProperty.class).get());
+ processedDAG.getIncomingEdgesOf(vertex12).forEach(irEdge -> {
+ assertEquals(InterTaskDataStoreProperty.Value.LocalFileStore, irEdge.getPropertyValue(InterTaskDataStoreProperty.class).get());
+ assertEquals(DataFlowModelProperty.Value.Pull, irEdge.getPropertyValue(DataFlowModelProperty.class).get());
+ });
+
+ final IRVertex vertex14 = processedDAG.getTopologicalSort().get(12);
+ assertEquals(ExecutorPlacementProperty.RESERVED, vertex14.getPropertyValue(ExecutorPlacementProperty.class).get());
+ processedDAG.getIncomingEdgesOf(vertex14).forEach(irEdge -> {
+ assertEquals(InterTaskDataStoreProperty.Value.LocalFileStore, irEdge.getPropertyValue(InterTaskDataStoreProperty.class).get());
+ assertEquals(DataFlowModelProperty.Value.Push, irEdge.getPropertyValue(DataFlowModelProperty.class).get());
+ });
+ }
+}
diff --git a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePassTest.java b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/SailfishPassTest.java
similarity index 58%
rename from compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePassTest.java
rename to compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/SailfishPassTest.java
index d97816c..e49d31a 100644
--- a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/LargeShuffleCompositePassTest.java
+++ b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/SailfishPassTest.java
@@ -32,11 +32,11 @@ import org.powermock.modules.junit4.PowerMockRunner;
import static org.junit.Assert.assertEquals;
/**
- * Test {@link LargeShuffleCompositePass}.
+ * Test {@link SailfishPass}.
*/
@RunWith(PowerMockRunner.class)
@PrepareForTest(JobLauncher.class)
-public class LargeShuffleCompositePassTest {
+public class SailfishPassTest {
private DAG<IRVertex, IREdge> compiledDAG;
@Before
@@ -45,44 +45,44 @@ public class LargeShuffleCompositePassTest {
}
@Test
- public void testLargeShuffle() {
- final DAG<IRVertex, IREdge> processedDAG = new LargeShuffleCompositePass().apply(compiledDAG);
+ public void testSailfish() {
+ final DAG<IRVertex, IREdge> processedDAG = new SailfishPass().apply(compiledDAG);
processedDAG.getTopologicalSort().forEach(irVertex -> {
if (processedDAG.getIncomingEdgesOf(irVertex).stream().anyMatch(irEdge ->
- CommunicationPatternProperty.Value.Shuffle
- .equals(irEdge.getPropertyValue(CommunicationPatternProperty.class).get()))) {
+ DataCommunicationPatternProperty.Value.Shuffle
+ .equals(irEdge.getPropertyValue(DataCommunicationPatternProperty.class).get()))) {
// Relay vertex
processedDAG.getIncomingEdgesOf(irVertex).forEach(edgeToMerger -> {
- if (CommunicationPatternProperty.Value.Shuffle
- .equals(edgeToMerger.getPropertyValue(CommunicationPatternProperty.class).get())) {
- assertEquals(DataFlowProperty.Value.Push,
- edgeToMerger.getPropertyValue(DataFlowProperty.class).get());
- assertEquals(DataPersistenceProperty.Value.Discard,
- edgeToMerger.getPropertyValue(DataPersistenceProperty.class).get());
- assertEquals(DataStoreProperty.Value.SerializedMemoryStore,
- edgeToMerger.getPropertyValue(DataStoreProperty.class).get());
+ if (DataCommunicationPatternProperty.Value.Shuffle
+ .equals(edgeToMerger.getPropertyValue(DataCommunicationPatternProperty.class).get())) {
+ assertEquals(DataFlowModelProperty.Value.Push,
+ edgeToMerger.getPropertyValue(DataFlowModelProperty.class).get());
+ assertEquals(UsedDataHandlingProperty.Value.Discard,
+ edgeToMerger.getPropertyValue(UsedDataHandlingProperty.class).get());
+ assertEquals(InterTaskDataStoreProperty.Value.SerializedMemoryStore,
+ edgeToMerger.getPropertyValue(InterTaskDataStoreProperty.class).get());
assertEquals(BytesDecoderFactory.of(),
edgeToMerger.getPropertyValue(DecoderProperty.class).get());
} else {
- assertEquals(DataFlowProperty.Value.Pull,
- edgeToMerger.getPropertyValue(DataFlowProperty.class).get());
+ assertEquals(DataFlowModelProperty.Value.Pull,
+ edgeToMerger.getPropertyValue(DataFlowModelProperty.class).get());
}
});
processedDAG.getOutgoingEdgesOf(irVertex).forEach(edgeFromMerger -> {
- assertEquals(DataFlowProperty.Value.Pull,
- edgeFromMerger.getPropertyValue(DataFlowProperty.class).get());
- assertEquals(CommunicationPatternProperty.Value.OneToOne,
- edgeFromMerger.getPropertyValue(CommunicationPatternProperty.class).get());
- assertEquals(DataStoreProperty.Value.LocalFileStore,
- edgeFromMerger.getPropertyValue(DataStoreProperty.class).get());
+ assertEquals(DataFlowModelProperty.Value.Pull,
+ edgeFromMerger.getPropertyValue(DataFlowModelProperty.class).get());
+ assertEquals(DataCommunicationPatternProperty.Value.OneToOne,
+ edgeFromMerger.getPropertyValue(DataCommunicationPatternProperty.class).get());
+ assertEquals(InterTaskDataStoreProperty.Value.LocalFileStore,
+ edgeFromMerger.getPropertyValue(InterTaskDataStoreProperty.class).get());
assertEquals(BytesEncoderFactory.of(),
edgeFromMerger.getPropertyValue(EncoderProperty.class).get());
});
} else {
// Non merger vertex.
processedDAG.getIncomingEdgesOf(irVertex).forEach(irEdge -> {
- assertEquals(DataFlowProperty.Value.Pull, irEdge.getPropertyValue(DataFlowProperty.class).get());
+ assertEquals(DataFlowModelProperty.Value.Pull, irEdge.getPropertyValue(DataFlowModelProperty.class).get());
});
}
});
diff --git a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/TransientResourceCompositePassTest.java b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/TransientResourceCompositePassTest.java
deleted file mode 100644
index 4d4649d..0000000
--- a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/composite/TransientResourceCompositePassTest.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Copyright (C) 2018 Seoul National University
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.snu.nemo.compiler.optimizer.pass.compiletime.composite;
-
-import edu.snu.nemo.client.JobLauncher;
-import edu.snu.nemo.common.dag.DAG;
-import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
-import edu.snu.nemo.common.ir.vertex.IRVertex;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
-import edu.snu.nemo.compiler.CompilerTestUtil;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.TransientResourceDataStorePass;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.TransientResourcePriorityPass;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Test {@link TransientResourcePriorityPass} and {@link TransientResourceDataStorePass}.
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(JobLauncher.class)
-public class TransientResourceCompositePassTest {
- private DAG<IRVertex, IREdge> compiledDAG;
-
- @Before
- public void setUp() throws Exception {
- compiledDAG = CompilerTestUtil.compileALSDAG();
- }
-
- @Test
- public void testTransientResourcePass() throws Exception {
- final DAG<IRVertex, IREdge> processedDAG = new TransientResourceCompositePass().apply(compiledDAG);
-
- final IRVertex vertex1 = processedDAG.getTopologicalSort().get(0);
- assertEquals(ResourcePriorityProperty.TRANSIENT, vertex1.getPropertyValue(ResourcePriorityProperty.class).get());
-
- final IRVertex vertex5 = processedDAG.getTopologicalSort().get(1);
- assertEquals(ResourcePriorityProperty.TRANSIENT, vertex5.getPropertyValue(ResourcePriorityProperty.class).get());
- processedDAG.getIncomingEdgesOf(vertex5).forEach(irEdge -> {
- assertEquals(DataStoreProperty.Value.MemoryStore, irEdge.getPropertyValue(DataStoreProperty.class).get());
- assertEquals(DataFlowProperty.Value.Pull, irEdge.getPropertyValue(DataFlowProperty.class).get());
- });
-
- final IRVertex vertex6 = processedDAG.getTopologicalSort().get(2);
- assertEquals(ResourcePriorityProperty.RESERVED, vertex6.getPropertyValue(ResourcePriorityProperty.class).get());
- processedDAG.getIncomingEdgesOf(vertex6).forEach(irEdge -> {
- assertEquals(DataStoreProperty.Value.LocalFileStore, irEdge.getPropertyValue(DataStoreProperty.class).get());
- assertEquals(DataFlowProperty.Value.Push, irEdge.getPropertyValue(DataFlowProperty.class).get());
- });
-
- final IRVertex vertex4 = processedDAG.getTopologicalSort().get(6);
- assertEquals(ResourcePriorityProperty.RESERVED, vertex4.getPropertyValue(ResourcePriorityProperty.class).get());
- processedDAG.getIncomingEdgesOf(vertex4).forEach(irEdge -> {
- assertEquals(DataStoreProperty.Value.MemoryStore, irEdge.getPropertyValue(DataStoreProperty.class).get());
- assertEquals(DataFlowProperty.Value.Pull, irEdge.getPropertyValue(DataFlowProperty.class).get());
- });
-
- final IRVertex vertex12 = processedDAG.getTopologicalSort().get(10);
- assertEquals(ResourcePriorityProperty.TRANSIENT, vertex12.getPropertyValue(ResourcePriorityProperty.class).get());
- processedDAG.getIncomingEdgesOf(vertex12).forEach(irEdge -> {
- assertEquals(DataStoreProperty.Value.LocalFileStore, irEdge.getPropertyValue(DataStoreProperty.class).get());
- assertEquals(DataFlowProperty.Value.Pull, irEdge.getPropertyValue(DataFlowProperty.class).get());
- });
-
- final IRVertex vertex14 = processedDAG.getTopologicalSort().get(12);
- assertEquals(ResourcePriorityProperty.RESERVED, vertex14.getPropertyValue(ResourcePriorityProperty.class).get());
- processedDAG.getIncomingEdgesOf(vertex14).forEach(irEdge -> {
- assertEquals(DataStoreProperty.Value.LocalFileStore, irEdge.getPropertyValue(DataStoreProperty.class).get());
- assertEquals(DataFlowProperty.Value.Push, irEdge.getPropertyValue(DataFlowProperty.class).get());
- });
- }
-}
diff --git a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPassTest.java b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPassTest.java
index cfb9a58..1f8b479 100644
--- a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPassTest.java
+++ b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/CommonSubexpressionEliminationPassTest.java
@@ -18,7 +18,7 @@ package edu.snu.nemo.compiler.optimizer.pass.compiletime.reshaping;
import edu.snu.nemo.client.JobLauncher;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
import edu.snu.nemo.common.dag.DAGBuilder;
@@ -56,16 +56,16 @@ public class CommonSubexpressionEliminationPassTest {
final DAGBuilder<IRVertex, IREdge> dagBuilder = new DAGBuilder<>();
dagNotToOptimize = dagBuilder.addVertex(source).addVertex(map1).addVertex(groupByKey).addVertex(combine)
.addVertex(map2)
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, source, map1))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.Shuffle, map1, groupByKey))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, groupByKey, combine))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, combine, map2))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, source, map1))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.Shuffle, map1, groupByKey))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, groupByKey, combine))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, combine, map2))
.build();
dagToOptimize = dagBuilder.addVertex(map1clone).addVertex(groupByKey2).addVertex(combine2).addVertex(map22)
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, source, map1clone))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.Shuffle, map1clone, groupByKey2))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, groupByKey2, combine2))
- .connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, combine2, map22))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, source, map1clone))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.Shuffle, map1clone, groupByKey2))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, groupByKey2, combine2))
+ .connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, combine2, map22))
.build();
}
diff --git a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopFusionPassTest.java b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopFusionPassTest.java
index 8e94779..2a784cf 100644
--- a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopFusionPassTest.java
+++ b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopFusionPassTest.java
@@ -19,7 +19,7 @@ import edu.snu.nemo.client.JobLauncher;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DecoderProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.EncoderProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
@@ -111,14 +111,14 @@ public class LoopFusionPassTest {
final LoopVertex loopVertexToFollow) {
builder.addVertex(loopVertexToFollow);
loopVertexToFollow.getIterativeIncomingEdges().values().forEach(irEdges -> irEdges.forEach(irEdge -> {
- final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
vertexToBeFollowed, loopVertexToFollow);
newIREdge.setProperty(EncoderProperty.of(irEdge.getPropertyValue(EncoderProperty.class).get()));
newIREdge.setProperty(DecoderProperty.of(irEdge.getPropertyValue(DecoderProperty.class).get()));
builder.connectVertices(newIREdge);
}));
loopVertexToFollow.getNonIterativeIncomingEdges().values().forEach(irEdges -> irEdges.forEach(irEdge -> {
- final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(CommunicationPatternProperty.class).get(),
+ final IREdge newIREdge = new IREdge(irEdge.getPropertyValue(DataCommunicationPatternProperty.class).get(),
irEdge.getSrc(), loopVertexToFollow);
newIREdge.setProperty(EncoderProperty.of(irEdge.getPropertyValue(EncoderProperty.class).get()));
newIREdge.setProperty(DecoderProperty.of(irEdge.getPropertyValue(DecoderProperty.class).get()));
diff --git a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopInvariantCodeMotionPassTest.java b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopInvariantCodeMotionPassTest.java
index 54f7933..1159921 100644
--- a/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopInvariantCodeMotionPassTest.java
+++ b/compiler/test/src/test/java/edu/snu/nemo/compiler/optimizer/pass/compiletime/reshaping/LoopInvariantCodeMotionPassTest.java
@@ -19,7 +19,7 @@ import edu.snu.nemo.client.JobLauncher;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DecoderProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.EncoderProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
@@ -90,7 +90,7 @@ public class LoopInvariantCodeMotionPassTest {
final Optional<IREdge> incomingEdge = newDAGIncomingEdge.stream().findFirst();
assertTrue(incomingEdge.isPresent());
final IREdge newIREdge = new IREdge(incomingEdge.get().getPropertyValue(
- CommunicationPatternProperty.class).get(), incomingEdge.get().getSrc(), alsLoop);
+ DataCommunicationPatternProperty.class).get(), incomingEdge.get().getSrc(), alsLoop);
newIREdge.setProperty(EncoderProperty.of(incomingEdge.get().getPropertyValue(EncoderProperty.class).get()));
newIREdge.setProperty(DecoderProperty.of(incomingEdge.get().getPropertyValue(DecoderProperty.class).get()));
builder.connectVertices(newIREdge);
diff --git a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/AlternatingLeastSquareITCase.java b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/AlternatingLeastSquareITCase.java
index 27331e9..a5b1f39 100644
--- a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/AlternatingLeastSquareITCase.java
+++ b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/AlternatingLeastSquareITCase.java
@@ -19,7 +19,8 @@ import edu.snu.nemo.client.JobLauncher;
import edu.snu.nemo.common.test.ArgBuilder;
import edu.snu.nemo.common.test.ExampleTestUtil;
import edu.snu.nemo.compiler.optimizer.policy.DefaultPolicy;
-import edu.snu.nemo.examples.beam.policy.TransientResourcePolicyParallelismTen;
+import edu.snu.nemo.examples.beam.policy.PadoPolicyParallelismFive;
+import edu.snu.nemo.examples.beam.policy.PadoPolicyParallelismTen;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -73,12 +74,12 @@ public final class AlternatingLeastSquareITCase {
}
@Test (timeout = TIMEOUT)
- public void testTransientResourceWithPoison() throws Exception {
+ public void testPadoWithPoison() throws Exception {
JobLauncher.main(builder
.addResourceJson(poisonedResource)
- .addJobId(AlternatingLeastSquareITCase.class.getSimpleName() + "_transient_poisoned")
+ .addJobId(AlternatingLeastSquareITCase.class.getSimpleName() + "_pado_poisoned")
.addMaxTaskAttempt(Integer.MAX_VALUE)
- .addOptimizationPolicy(TransientResourcePolicyParallelismTen.class.getCanonicalName())
+ .addOptimizationPolicy(PadoPolicyParallelismTen.class.getCanonicalName())
.build());
}
}
diff --git a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/BroadcastITCase.java b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/BroadcastITCase.java
index d813640..b49bc75 100644
--- a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/BroadcastITCase.java
+++ b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/BroadcastITCase.java
@@ -19,7 +19,7 @@ import edu.snu.nemo.client.JobLauncher;
import edu.snu.nemo.common.test.ArgBuilder;
import edu.snu.nemo.common.test.ExampleTestUtil;
import edu.snu.nemo.examples.beam.policy.DefaultPolicyParallelismFive;
-import edu.snu.nemo.examples.beam.policy.TransientResourcePolicyParallelismFive;
+import edu.snu.nemo.examples.beam.policy.PadoPolicyParallelismFive;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -70,10 +70,10 @@ public final class BroadcastITCase {
}
@Test (timeout = TIMEOUT)
- public void testTransientResource() throws Exception {
+ public void testPado() throws Exception {
JobLauncher.main(builder
- .addJobId(BroadcastITCase.class.getSimpleName() + "_transient")
- .addOptimizationPolicy(TransientResourcePolicyParallelismFive.class.getCanonicalName())
+ .addJobId(BroadcastITCase.class.getSimpleName() + "_pado")
+ .addOptimizationPolicy(PadoPolicyParallelismFive.class.getCanonicalName())
.build());
}
}
diff --git a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/NetworkTraceAnalysisITCase.java b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/NetworkTraceAnalysisITCase.java
index d733f66..f5e344c 100644
--- a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/NetworkTraceAnalysisITCase.java
+++ b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/NetworkTraceAnalysisITCase.java
@@ -19,8 +19,8 @@ import edu.snu.nemo.client.JobLauncher;
import edu.snu.nemo.common.test.ArgBuilder;
import edu.snu.nemo.common.test.ExampleTestUtil;
import edu.snu.nemo.examples.beam.policy.DefaultPolicyParallelismFive;
-import edu.snu.nemo.examples.beam.policy.TransientResourcePolicyParallelismFive;
-import edu.snu.nemo.examples.beam.policy.LargeShufflePolicyParallelismFive;
+import edu.snu.nemo.examples.beam.policy.PadoPolicyParallelismFive;
+import edu.snu.nemo.examples.beam.policy.SailfishPolicyParallelismFive;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -70,18 +70,18 @@ public final class NetworkTraceAnalysisITCase {
}
@Test (timeout = TIMEOUT)
- public void testLargeShuffle() throws Exception {
+ public void testSailfish() throws Exception {
JobLauncher.main(builder
- .addJobId(NetworkTraceAnalysisITCase.class.getSimpleName() + "_largeshuffle")
- .addOptimizationPolicy(LargeShufflePolicyParallelismFive.class.getCanonicalName())
+ .addJobId(NetworkTraceAnalysisITCase.class.getSimpleName() + "_sailfish")
+ .addOptimizationPolicy(SailfishPolicyParallelismFive.class.getCanonicalName())
.build());
}
@Test (timeout = TIMEOUT)
- public void testTransientResource() throws Exception {
+ public void testPado() throws Exception {
JobLauncher.main(builder
- .addJobId(NetworkTraceAnalysisITCase.class.getSimpleName() + "_transient")
- .addOptimizationPolicy(TransientResourcePolicyParallelismFive.class.getCanonicalName())
+ .addJobId(NetworkTraceAnalysisITCase.class.getSimpleName() + "_pado")
+ .addOptimizationPolicy(PadoPolicyParallelismFive.class.getCanonicalName())
.build());
}
}
diff --git a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/WordCountITCase.java b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/WordCountITCase.java
index 6527e26..0ffba2e 100644
--- a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/WordCountITCase.java
+++ b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/WordCountITCase.java
@@ -70,29 +70,29 @@ public final class WordCountITCase {
}
@Test (timeout = TIMEOUT)
- public void testLargeShuffle() throws Exception {
+ public void testSailfish() throws Exception {
JobLauncher.main(builder
.addResourceJson(executorResourceFileName)
- .addJobId(WordCountITCase.class.getSimpleName() + "_largeshuffle")
- .addOptimizationPolicy(LargeShufflePolicyParallelismFive.class.getCanonicalName())
+ .addJobId(WordCountITCase.class.getSimpleName() + "_sailfish")
+ .addOptimizationPolicy(SailfishPolicyParallelismFive.class.getCanonicalName())
.build());
}
@Test (timeout = TIMEOUT)
- public void testLargeShuffleInOneExecutor() throws Exception {
+ public void testSailfishInOneExecutor() throws Exception {
JobLauncher.main(builder
.addResourceJson(oneExecutorResourceFileName)
- .addJobId(WordCountITCase.class.getSimpleName() + "_largeshuffleInOneExecutor")
- .addOptimizationPolicy(LargeShufflePolicyParallelismFive.class.getCanonicalName())
+ .addJobId(WordCountITCase.class.getSimpleName() + "_sailfishInOneExecutor")
+ .addOptimizationPolicy(SailfishPolicyParallelismFive.class.getCanonicalName())
.build());
}
@Test (timeout = TIMEOUT)
- public void testTransientResource() throws Exception {
+ public void testPado() throws Exception {
JobLauncher.main(builder
.addResourceJson(executorResourceFileName)
- .addJobId(WordCountITCase.class.getSimpleName() + "_transient")
- .addOptimizationPolicy(TransientResourcePolicyParallelismFive.class.getCanonicalName())
+ .addJobId(WordCountITCase.class.getSimpleName() + "_pado")
+ .addOptimizationPolicy(PadoPolicyParallelismFive.class.getCanonicalName())
.build());
}
}
diff --git a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/LargeShufflePolicyParallelismFive.java b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/PadoPolicyParallelismFive.java
similarity index 76%
rename from examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/LargeShufflePolicyParallelismFive.java
rename to examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/PadoPolicyParallelismFive.java
index b4c6339..d1c8de4 100644
--- a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/LargeShufflePolicyParallelismFive.java
+++ b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/PadoPolicyParallelismFive.java
@@ -16,21 +16,20 @@
package edu.snu.nemo.examples.beam.policy;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
+import edu.snu.nemo.compiler.optimizer.policy.PadoPolicy;
import edu.snu.nemo.compiler.optimizer.policy.Policy;
-import edu.snu.nemo.compiler.optimizer.policy.LargeShufflePolicy;
import edu.snu.nemo.runtime.common.optimizer.pass.runtime.RuntimePass;
import java.util.List;
/**
- * A large shuffle policy with fixed parallelism 5 for tests.
+ * A pado policy with fixed parallelism 5 for tests.
*/
-public final class LargeShufflePolicyParallelismFive implements Policy {
+public final class PadoPolicyParallelismFive implements Policy {
private final Policy policy;
- public LargeShufflePolicyParallelismFive() {
- this.policy =
- PolicyTestUtil.overwriteParallelism(5, LargeShufflePolicy.class.getCanonicalName());
+ public PadoPolicyParallelismFive() {
+ this.policy = PolicyTestUtil.overwriteParallelism(5, PadoPolicy.class.getCanonicalName());
}
@Override
diff --git a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/TransientResourcePolicyParallelismTen.java b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/PadoPolicyParallelismTen.java
similarity index 75%
rename from examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/TransientResourcePolicyParallelismTen.java
rename to examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/PadoPolicyParallelismTen.java
index 4227b80..53e82cc 100644
--- a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/TransientResourcePolicyParallelismTen.java
+++ b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/PadoPolicyParallelismTen.java
@@ -16,21 +16,20 @@
package edu.snu.nemo.examples.beam.policy;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
-import edu.snu.nemo.compiler.optimizer.policy.TransientResourcePolicy;
+import edu.snu.nemo.compiler.optimizer.policy.PadoPolicy;
import edu.snu.nemo.compiler.optimizer.policy.Policy;
import edu.snu.nemo.runtime.common.optimizer.pass.runtime.RuntimePass;
import java.util.List;
/**
- * A transient resource policy with fixed parallelism 10 for tests.
+ * A pado policy with fixed parallelism 10 for tests.
*/
-public final class TransientResourcePolicyParallelismTen implements Policy {
+public final class PadoPolicyParallelismTen implements Policy {
private final Policy policy;
- public TransientResourcePolicyParallelismTen() {
- this.policy =
- PolicyTestUtil.overwriteParallelism(10, TransientResourcePolicy.class.getCanonicalName());
+ public PadoPolicyParallelismTen() {
+ this.policy = PolicyTestUtil.overwriteParallelism(10, PadoPolicy.class.getCanonicalName());
}
@Override
diff --git a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/TransientResourcePolicyParallelismFive.java b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/SailfishPolicyParallelismFive.java
similarity index 75%
rename from examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/TransientResourcePolicyParallelismFive.java
rename to examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/SailfishPolicyParallelismFive.java
index 6760917..55754b4 100644
--- a/examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/TransientResourcePolicyParallelismFive.java
+++ b/examples/beam/src/test/java/edu/snu/nemo/examples/beam/policy/SailfishPolicyParallelismFive.java
@@ -16,21 +16,20 @@
package edu.snu.nemo.examples.beam.policy;
import edu.snu.nemo.compiler.optimizer.pass.compiletime.CompileTimePass;
-import edu.snu.nemo.compiler.optimizer.policy.TransientResourcePolicy;
import edu.snu.nemo.compiler.optimizer.policy.Policy;
+import edu.snu.nemo.compiler.optimizer.policy.SailfishPolicy;
import edu.snu.nemo.runtime.common.optimizer.pass.runtime.RuntimePass;
import java.util.List;
/**
- * A transient resource policy with fixed parallelism 5 for tests.
+ * A Sailfish policy with fixed parallelism 5 for tests.
*/
-public final class TransientResourcePolicyParallelismFive implements Policy {
+public final class SailfishPolicyParallelismFive implements Policy {
private final Policy policy;
- public TransientResourcePolicyParallelismFive() {
- this.policy =
- PolicyTestUtil.overwriteParallelism(5, TransientResourcePolicy.class.getCanonicalName());
+ public SailfishPolicyParallelismFive() {
+ this.policy = PolicyTestUtil.overwriteParallelism(5, SailfishPolicy.class.getCanonicalName());
}
@Override
diff --git a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/plan/PhysicalPlanGenerator.java b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/plan/PhysicalPlanGenerator.java
index fa95926..13d8f94 100644
--- a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/plan/PhysicalPlanGenerator.java
+++ b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/plan/PhysicalPlanGenerator.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.runtime.common.plan;
import edu.snu.nemo.common.ir.Readable;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DuplicateEdgeGroupProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DuplicateEdgeGroupPropertyValue;
import edu.snu.nemo.common.ir.executionproperty.ExecutionPropertyMap;
@@ -276,7 +276,7 @@ public final class PhysicalPlanGenerator implements Function<DAG<IRVertex, IREdg
Integer newScheduleGroup = null;
for (final StageEdge stageEdge : dag.getIncomingEdgesOf(destination)) {
final Stage source = stageEdge.getSrc();
- if (stageEdge.getDataFlowModel() != DataFlowProperty.Value.Pull) {
+ if (stageEdge.getDataFlowModel() != DataFlowModelProperty.Value.Pull) {
if (scheduleGroup != null && source.getScheduleGroup() != scheduleGroup) {
throw new RuntimeException(String.format("Multiple Push inEdges from different ScheduleGroup: %d, %d",
scheduleGroup, source.getScheduleGroup()));
diff --git a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/plan/StageEdge.java b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/plan/StageEdge.java
index 98773ea..5750c2a 100644
--- a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/plan/StageEdge.java
+++ b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/plan/StageEdge.java
@@ -16,8 +16,8 @@
package edu.snu.nemo.runtime.common.plan;
import com.google.common.annotations.VisibleForTesting;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
import edu.snu.nemo.common.ir.executionproperty.EdgeExecutionProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.executionproperty.ExecutionPropertyMap;
@@ -50,14 +50,14 @@ public final class StageEdge extends RuntimeEdge<Stage> {
private Map<Integer, KeyRange> taskIdxToKeyRange;
/**
- * Value for {@link CommunicationPatternProperty}.
+ * Value for {@link DataCommunicationPatternProperty}.
*/
- private final CommunicationPatternProperty.Value dataCommunicationPatternValue;
+ private final DataCommunicationPatternProperty.Value dataCommunicationPatternValue;
/**
- * Value for {@link DataFlowProperty}.
+ * Value for {@link DataFlowModelProperty}.
*/
- private final DataFlowProperty.Value dataFlowModelValue;
+ private final DataFlowModelProperty.Value dataFlowModelValue;
/**
* Constructor.
@@ -86,12 +86,12 @@ public final class StageEdge extends RuntimeEdge<Stage> {
for (int taskIdx = 0; taskIdx < dstStage.getTaskIds().size(); taskIdx++) {
taskIdxToKeyRange.put(taskIdx, HashRange.of(taskIdx, taskIdx + 1, false));
}
- this.dataCommunicationPatternValue = edgeProperties.get(CommunicationPatternProperty.class)
+ this.dataCommunicationPatternValue = edgeProperties.get(DataCommunicationPatternProperty.class)
.orElseThrow(() -> new RuntimeException(String.format(
- "CommunicationPatternProperty not set for %s", runtimeEdgeId)));
- this.dataFlowModelValue = edgeProperties.get(DataFlowProperty.class)
+ "DataCommunicationPatternProperty not set for %s", runtimeEdgeId)));
+ this.dataFlowModelValue = edgeProperties.get(DataFlowModelProperty.class)
.orElseThrow(() -> new RuntimeException(String.format(
- "DataFlowProperty not set for %s", runtimeEdgeId)));
+ "DataFlowModelProperty not set for %s", runtimeEdgeId)));
}
/**
@@ -141,16 +141,16 @@ public final class StageEdge extends RuntimeEdge<Stage> {
}
/**
- * @return {@link CommunicationPatternProperty} value.
+ * @return {@link DataCommunicationPatternProperty} value.
*/
- public CommunicationPatternProperty.Value getDataCommunicationPattern() {
+ public DataCommunicationPatternProperty.Value getDataCommunicationPattern() {
return dataCommunicationPatternValue;
}
/**
- * @return {@link DataFlowProperty} value.
+ * @return {@link DataFlowModelProperty} value.
*/
- public DataFlowProperty.Value getDataFlowModel() {
+ public DataFlowModelProperty.Value getDataFlowModel() {
return dataFlowModelValue;
}
}
diff --git a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/plan/StagePartitioner.java b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/plan/StagePartitioner.java
index d6926e4..49553dc 100644
--- a/runtime/common/src/main/java/edu/snu/nemo/runtime/common/plan/StagePartitioner.java
+++ b/runtime/common/src/main/java/edu/snu/nemo/runtime/common/plan/StagePartitioner.java
@@ -17,7 +17,7 @@ package edu.snu.nemo.runtime.common.plan;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.executionproperty.VertexExecutionProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import net.jcip.annotations.ThreadSafe;
@@ -106,8 +106,8 @@ public final class StagePartitioner implements Function<DAG<IRVertex, IREdge>, M
return false;
}
// If the edge is not OneToOne, return false
- if (edge.getPropertyValue(CommunicationPatternProperty.class).get()
- != CommunicationPatternProperty.Value.OneToOne) {
+ if (edge.getPropertyValue(DataCommunicationPatternProperty.class).get()
+ != DataCommunicationPatternProperty.Value.OneToOne) {
return false;
}
// Return true if and only if the execution properties of the two vertices are compatible
diff --git a/runtime/common/src/test/java/edu/snu/nemo/runtime/common/plan/PhysicalPlanGeneratorTest.java b/runtime/common/src/test/java/edu/snu/nemo/runtime/common/plan/PhysicalPlanGeneratorTest.java
index 8f3c957..2427232 100644
--- a/runtime/common/src/test/java/edu/snu/nemo/runtime/common/plan/PhysicalPlanGeneratorTest.java
+++ b/runtime/common/src/test/java/edu/snu/nemo/runtime/common/plan/PhysicalPlanGeneratorTest.java
@@ -18,8 +18,8 @@ package edu.snu.nemo.runtime.common.plan;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataFlowModelProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
import edu.snu.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
@@ -52,8 +52,8 @@ public final class PhysicalPlanGeneratorTest {
final DAG<IRVertex, IREdge> irDAG = new DAGBuilder<IRVertex, IREdge>()
.addVertex(v0)
.addVertex(v1)
- .connectVertices(newIREdge(v0, v1, CommunicationPatternProperty.Value.OneToOne,
- DataFlowProperty.Value.Pull))
+ .connectVertices(newIREdge(v0, v1, DataCommunicationPatternProperty.Value.OneToOne,
+ DataFlowModelProperty.Value.Pull))
.buildWithoutSourceSinkCheck();
final DAG<Stage, StageEdge> stageDAG = physicalPlanGenerator.apply(irDAG);
@@ -72,10 +72,10 @@ public final class PhysicalPlanGeneratorTest {
}
private static final IREdge newIREdge(final IRVertex src, final IRVertex dst,
- final CommunicationPatternProperty.Value communicationPattern,
- final DataFlowProperty.Value dataFlowModel) {
+ final DataCommunicationPatternProperty.Value communicationPattern,
+ final DataFlowModelProperty.Value dataFlowModel) {
final IREdge irEdge = new IREdge(communicationPattern, src, dst);
- irEdge.getExecutionProperties().put(DataFlowProperty.of(dataFlowModel));
+ irEdge.getExecutionProperties().put(DataFlowModelProperty.of(dataFlowModel));
return irEdge;
}
}
diff --git a/runtime/common/src/test/java/edu/snu/nemo/runtime/common/plan/StagePartitionerTest.java b/runtime/common/src/test/java/edu/snu/nemo/runtime/common/plan/StagePartitionerTest.java
index 48a47d0..b9f6c74 100644
--- a/runtime/common/src/test/java/edu/snu/nemo/runtime/common/plan/StagePartitionerTest.java
+++ b/runtime/common/src/test/java/edu/snu/nemo/runtime/common/plan/StagePartitionerTest.java
@@ -17,12 +17,17 @@ package edu.snu.nemo.runtime.common.plan;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.executionproperty.VertexExecutionProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
-import edu.snu.nemo.common.ir.vertex.executionproperty.*;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.DynamicOptimizationProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ScheduleGroupProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.AdditionalTagOutputProperty;
+import edu.snu.nemo.common.ir.vertex.transform.Transform;
+import edu.snu.nemo.common.test.EmptyComponents;
import org.apache.reef.tang.Tang;
import org.apache.reef.tang.exceptions.InjectionException;
import org.junit.Before;
@@ -75,7 +80,7 @@ public final class StagePartitionerTest {
final IRVertex v1 = newVertex(5, 0, Collections.emptyList());
dagBuilder.addVertex(v0);
dagBuilder.addVertex(v1);
- dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v0, v1));
+ dagBuilder.connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v0, v1));
final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
assertEquals(0, (int) partitioning.get(v0));
assertEquals(0, (int) partitioning.get(v1));
@@ -91,7 +96,7 @@ public final class StagePartitionerTest {
final IRVertex v1 = newVertex(1, 0, Collections.emptyList());
dagBuilder.addVertex(v0);
dagBuilder.addVertex(v1);
- dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v0, v1));
+ dagBuilder.connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v0, v1));
final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
assertNotEquals(partitioning.get(v0), partitioning.get(v1));
}
@@ -106,7 +111,7 @@ public final class StagePartitionerTest {
final IRVertex v1 = newVertex(1, 1, Collections.emptyList());
dagBuilder.addVertex(v0);
dagBuilder.addVertex(v1);
- dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v0, v1));
+ dagBuilder.connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v0, v1));
final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
assertNotEquals(partitioning.get(v0), partitioning.get(v1));
}
@@ -121,7 +126,7 @@ public final class StagePartitionerTest {
final IRVertex v1 = newVertex(1, 0, Collections.emptyList());
dagBuilder.addVertex(v0);
dagBuilder.addVertex(v1);
- dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.Shuffle, v0, v1));
+ dagBuilder.connectVertices(new IREdge(DataCommunicationPatternProperty.Value.Shuffle, v0, v1));
final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
assertNotEquals(partitioning.get(v0), partitioning.get(v1));
}
@@ -133,11 +138,11 @@ public final class StagePartitionerTest {
public void testSplitByOtherProperty() {
final DAGBuilder<IRVertex, IREdge> dagBuilder = new DAGBuilder<>();
final IRVertex v0 = newVertex(1, 0,
- Arrays.asList(ResourcePriorityProperty.of(ResourcePriorityProperty.RESERVED)));
+ Arrays.asList(ExecutorPlacementProperty.of(ExecutorPlacementProperty.RESERVED)));
final IRVertex v1 = newVertex(1, 0, Collections.emptyList());
dagBuilder.addVertex(v0);
dagBuilder.addVertex(v1);
- dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v0, v1));
+ dagBuilder.connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v0, v1));
final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
assertNotEquals(partitioning.get(v0), partitioning.get(v1));
}
@@ -153,7 +158,7 @@ public final class StagePartitionerTest {
final IRVertex v1 = newVertex(1, 0, Collections.emptyList());
dagBuilder.addVertex(v0);
dagBuilder.addVertex(v1);
- dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v0, v1));
+ dagBuilder.connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v0, v1));
final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
assertEquals(0, (int) partitioning.get(v0));
assertEquals(0, (int) partitioning.get(v1));
@@ -171,8 +176,8 @@ public final class StagePartitionerTest {
dagBuilder.addVertex(v0);
dagBuilder.addVertex(v1);
dagBuilder.addVertex(v2);
- dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v0, v2));
- dagBuilder.connectVertices(new IREdge(CommunicationPatternProperty.Value.OneToOne, v1, v2));
+ dagBuilder.connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v0, v2));
+ dagBuilder.connectVertices(new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v1, v2));
final Map<IRVertex, Integer> partitioning = stagePartitioner.apply(dagBuilder.buildWithoutSourceSinkCheck());
assertNotEquals(partitioning.get(v0), partitioning.get(v1));
assertNotEquals(partitioning.get(v1), partitioning.get(v2));
diff --git a/runtime/driver/src/main/java/edu/snu/nemo/driver/NemoDriver.java b/runtime/driver/src/main/java/edu/snu/nemo/driver/NemoDriver.java
index 14bffb5..ca232e0 100644
--- a/runtime/driver/src/main/java/edu/snu/nemo/driver/NemoDriver.java
+++ b/runtime/driver/src/main/java/edu/snu/nemo/driver/NemoDriver.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.driver;
import edu.snu.nemo.common.ir.IdManager;
-import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.ResourceSitePass;
+import edu.snu.nemo.compiler.optimizer.pass.compiletime.annotating.NodeNamesAssignmentPass;
import edu.snu.nemo.conf.JobConf;
import edu.snu.nemo.runtime.common.RuntimeIdGenerator;
import edu.snu.nemo.runtime.common.comm.ControlMessage;
@@ -103,7 +103,7 @@ public final class NemoDriver {
this.handler = new RemoteClientMessageLoggingHandler(client);
this.clientRPC = clientRPC;
// TODO #69: Support job-wide execution property
- ResourceSitePass.setBandwidthSpecificationString(bandwidthString);
+ NodeNamesAssignmentPass.setBandwidthSpecificationString(bandwidthString);
clientRPC.registerHandler(ControlMessage.ClientToDriverMessageType.LaunchDAG, message ->
startSchedulingUserDAG(message.getLaunchDAG().getDag()));
clientRPC.registerHandler(ControlMessage.ClientToDriverMessageType.DriverShutdown, message -> shutdown());
diff --git a/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/data/BlockManagerWorker.java b/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/data/BlockManagerWorker.java
index ba0a71c..d6a3da7 100644
--- a/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/data/BlockManagerWorker.java
+++ b/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/data/BlockManagerWorker.java
@@ -21,8 +21,8 @@ import edu.snu.nemo.common.exception.BlockFetchException;
import edu.snu.nemo.common.exception.BlockWriteException;
import edu.snu.nemo.common.exception.UnsupportedBlockStoreException;
import edu.snu.nemo.common.exception.UnsupportedExecutionPropertyException;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataPersistenceProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.UsedDataHandlingProperty;
import edu.snu.nemo.conf.JobConf;
import edu.snu.nemo.runtime.common.comm.ControlMessage;
import edu.snu.nemo.runtime.common.comm.ControlMessage.ByteTransferContextDescriptor;
@@ -129,7 +129,7 @@ public final class BlockManagerWorker {
* @throws BlockWriteException for any error occurred while trying to create a block.
*/
public Block createBlock(final String blockId,
- final DataStoreProperty.Value blockStore) throws BlockWriteException {
+ final InterTaskDataStoreProperty.Value blockStore) throws BlockWriteException {
final BlockStore store = getBlockStore(blockStore);
return store.createBlock(blockId);
}
@@ -148,7 +148,7 @@ public final class BlockManagerWorker {
public CompletableFuture<DataUtil.IteratorWithNumBytes> readBlock(
final String blockId,
final String runtimeEdgeId,
- final DataStoreProperty.Value blockStore,
+ final InterTaskDataStoreProperty.Value blockStore,
final KeyRange keyRange) {
// Let's see if a remote worker has it
final CompletableFuture<ControlMessage.Message> blockLocationFuture =
@@ -234,19 +234,19 @@ public final class BlockManagerWorker {
* @param partitionSizeMap the map of partition keys and sizes to report.
* @param srcIRVertexId the IR vertex ID of the source task.
* @param expectedReadTotal the expected number of read for this block.
- * @param persistence how to handle the used block.
+ * @param usedDataHandling how to handle the used block.
*/
public void writeBlock(final Block block,
- final DataStoreProperty.Value blockStore,
+ final InterTaskDataStoreProperty.Value blockStore,
final boolean reportPartitionSizes,
final Map<Integer, Long> partitionSizeMap,
final String srcIRVertexId,
final int expectedReadTotal,
- final DataPersistenceProperty.Value persistence) {
+ final UsedDataHandlingProperty.Value usedDataHandling) {
final String blockId = block.getId();
LOG.info("CommitBlock: {}", blockId);
- switch (persistence) {
+ switch (usedDataHandling) {
case Discard:
blockToRemainingRead.put(block.getId(), new AtomicInteger(expectedReadTotal));
break;
@@ -265,7 +265,7 @@ public final class BlockManagerWorker {
.setBlockId(blockId)
.setState(ControlMessage.BlockStateFromExecutor.AVAILABLE);
- if (DataStoreProperty.Value.GlusterFileStore.equals(blockStore)) {
+ if (InterTaskDataStoreProperty.Value.GlusterFileStore.equals(blockStore)) {
blockStateChangedMsgBuilder.setLocation(REMOTE_FILE_STORE);
} else {
blockStateChangedMsgBuilder.setLocation(executorId);
@@ -311,7 +311,7 @@ public final class BlockManagerWorker {
* @param blockStore the store which contains the block.
*/
public void removeBlock(final String blockId,
- final DataStoreProperty.Value blockStore) {
+ final InterTaskDataStoreProperty.Value blockStore) {
LOG.info("RemoveBlock: {}", blockId);
final BlockStore store = getBlockStore(blockStore);
final boolean deleted = store.deleteBlock(blockId);
@@ -323,7 +323,7 @@ public final class BlockManagerWorker {
.setBlockId(blockId)
.setState(ControlMessage.BlockStateFromExecutor.NOT_AVAILABLE);
- if (DataStoreProperty.Value.GlusterFileStore.equals(blockStore)) {
+ if (InterTaskDataStoreProperty.Value.GlusterFileStore.equals(blockStore)) {
blockStateChangedMsgBuilder.setLocation(REMOTE_FILE_STORE);
} else {
blockStateChangedMsgBuilder.setLocation(executorId);
@@ -355,7 +355,7 @@ public final class BlockManagerWorker {
public void onOutputContext(final ByteOutputContext outputContext) throws InvalidProtocolBufferException {
final ByteTransferContextDescriptor descriptor = ByteTransferContextDescriptor.PARSER
.parseFrom(outputContext.getContextDescriptor());
- final DataStoreProperty.Value blockStore = convertBlockStore(descriptor.getBlockStore());
+ final InterTaskDataStoreProperty.Value blockStore = convertBlockStore(descriptor.getBlockStore());
final String blockId = descriptor.getBlockId();
final KeyRange keyRange = SerializationUtils.deserialize(descriptor.getKeyRange().toByteArray());
@@ -365,8 +365,8 @@ public final class BlockManagerWorker {
try {
final Optional<Block> optionalBlock = getBlockStore(blockStore).readBlock(blockId);
if (optionalBlock.isPresent()) {
- if (DataStoreProperty.Value.LocalFileStore.equals(blockStore)
- || DataStoreProperty.Value.GlusterFileStore.equals(blockStore)) {
+ if (InterTaskDataStoreProperty.Value.LocalFileStore.equals(blockStore)
+ || InterTaskDataStoreProperty.Value.GlusterFileStore.equals(blockStore)) {
final List<FileArea> fileAreas = ((FileBlock) optionalBlock.get()).asFileAreas(keyRange);
for (final FileArea fileArea : fileAreas) {
try (ByteOutputContext.ByteOutputStream os = outputContext.newOutputStream()) {
@@ -381,7 +381,7 @@ public final class BlockManagerWorker {
}
}
}
- handleDataPersistence(blockStore, blockId);
+ handleUsedData(blockStore, blockId);
outputContext.close();
} else {
@@ -420,7 +420,7 @@ public final class BlockManagerWorker {
*/
private CompletableFuture<DataUtil.IteratorWithNumBytes> getDataFromLocalBlock(
final String blockId,
- final DataStoreProperty.Value blockStore,
+ final InterTaskDataStoreProperty.Value blockStore,
final KeyRange keyRange) {
final BlockStore store = getBlockStore(blockStore);
@@ -429,7 +429,7 @@ public final class BlockManagerWorker {
if (optionalBlock.isPresent()) {
final Iterable<NonSerializedPartition> partitions = optionalBlock.get().readPartitions(keyRange);
- handleDataPersistence(blockStore, blockId);
+ handleUsedData(blockStore, blockId);
// Block resides in this evaluator!
try {
@@ -463,8 +463,8 @@ public final class BlockManagerWorker {
* @param blockStore the store which contains the block.
* @param blockId the ID of the block.
*/
- private void handleDataPersistence(final DataStoreProperty.Value blockStore,
- final String blockId) {
+ private void handleUsedData(final InterTaskDataStoreProperty.Value blockStore,
+ final String blockId) {
final AtomicInteger remainingExpectedRead = blockToRemainingRead.get(blockId);
if (remainingExpectedRead != null) {
if (remainingExpectedRead.decrementAndGet() == 0) {
@@ -483,11 +483,11 @@ public final class BlockManagerWorker {
//////////////////////////////////////////////////////////// Converters
/**
- * Gets the {@link BlockStore} from annotated value of {@link DataStoreProperty}.
- * @param blockStore the annotated value of {@link DataStoreProperty}.
+ * Gets the {@link BlockStore} from annotated value of {@link InterTaskDataStoreProperty}.
+ * @param blockStore the annotated value of {@link InterTaskDataStoreProperty}.
* @return the block store.
*/
- private BlockStore getBlockStore(final DataStoreProperty.Value blockStore) {
+ private BlockStore getBlockStore(final InterTaskDataStoreProperty.Value blockStore) {
switch (blockStore) {
case MemoryStore:
return memoryStore;
@@ -506,10 +506,10 @@ public final class BlockManagerWorker {
/**
* Decodes BlockStore property from protocol buffer.
* @param blockStore property from protocol buffer
- * @return the corresponding {@link DataStoreProperty} value
+ * @return the corresponding {@link InterTaskDataStoreProperty} value
*/
private static ControlMessage.BlockStore convertBlockStore(
- final DataStoreProperty.Value blockStore) {
+ final InterTaskDataStoreProperty.Value blockStore) {
switch (blockStore) {
case MemoryStore:
return ControlMessage.BlockStore.MEMORY;
@@ -526,21 +526,21 @@ public final class BlockManagerWorker {
/**
- * Encodes {@link DataStoreProperty} value into protocol buffer property.
- * @param blockStoreType {@link DataStoreProperty} value
+ * Encodes {@link InterTaskDataStoreProperty} value into protocol buffer property.
+ * @param blockStoreType {@link InterTaskDataStoreProperty} value
* @return the corresponding {@link ControlMessage.BlockStore} value
*/
- private static DataStoreProperty.Value convertBlockStore(
+ private static InterTaskDataStoreProperty.Value convertBlockStore(
final ControlMessage.BlockStore blockStoreType) {
switch (blockStoreType) {
case MEMORY:
- return DataStoreProperty.Value.MemoryStore;
+ return InterTaskDataStoreProperty.Value.MemoryStore;
case SER_MEMORY:
- return DataStoreProperty.Value.SerializedMemoryStore;
+ return InterTaskDataStoreProperty.Value.SerializedMemoryStore;
case LOCAL_FILE:
- return DataStoreProperty.Value.LocalFileStore;
+ return InterTaskDataStoreProperty.Value.LocalFileStore;
case REMOTE_FILE:
- return DataStoreProperty.Value.GlusterFileStore;
+ return InterTaskDataStoreProperty.Value.GlusterFileStore;
default:
throw new UnsupportedBlockStoreException(new Exception("This block store is not yet supported"));
}
diff --git a/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/datatransfer/InputReader.java b/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/datatransfer/InputReader.java
index e9e563b..e417c82 100644
--- a/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/datatransfer/InputReader.java
+++ b/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/datatransfer/InputReader.java
@@ -16,8 +16,8 @@
package edu.snu.nemo.runtime.executor.datatransfer;
import com.google.common.annotations.VisibleForTesting;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DuplicateEdgeGroupProperty;
import edu.snu.nemo.common.ir.edge.executionproperty.DuplicateEdgeGroupPropertyValue;
import edu.snu.nemo.common.ir.vertex.IRVertex;
@@ -72,14 +72,14 @@ public final class InputReader extends DataTransfer {
* @return the read data.
*/
public List<CompletableFuture<DataUtil.IteratorWithNumBytes>> read() {
- final Optional<CommunicationPatternProperty.Value> comValue =
- runtimeEdge.getPropertyValue(CommunicationPatternProperty.class);
+ final Optional<DataCommunicationPatternProperty.Value> comValue =
+ runtimeEdge.getPropertyValue(DataCommunicationPatternProperty.class);
- if (comValue.get().equals(CommunicationPatternProperty.Value.OneToOne)) {
+ if (comValue.get().equals(DataCommunicationPatternProperty.Value.OneToOne)) {
return Collections.singletonList(readOneToOne());
- } else if (comValue.get().equals(CommunicationPatternProperty.Value.BroadCast)) {
+ } else if (comValue.get().equals(DataCommunicationPatternProperty.Value.BroadCast)) {
return readBroadcast();
- } else if (comValue.get().equals(CommunicationPatternProperty.Value.Shuffle)) {
+ } else if (comValue.get().equals(DataCommunicationPatternProperty.Value.Shuffle)) {
// If the dynamic optimization which detects data skew is enabled, read the data in the assigned range.
// TODO #492: Modularize the data communication pattern.
return readDataInRange();
@@ -90,15 +90,15 @@ public final class InputReader extends DataTransfer {
private CompletableFuture<DataUtil.IteratorWithNumBytes> readOneToOne() {
final String blockId = getBlockId(dstTaskIndex);
- final Optional<DataStoreProperty.Value> dataStoreProperty
- = runtimeEdge.getPropertyValue(DataStoreProperty.class);
+ final Optional<InterTaskDataStoreProperty.Value> dataStoreProperty
+ = runtimeEdge.getPropertyValue(InterTaskDataStoreProperty.class);
return blockManagerWorker.readBlock(blockId, getId(), dataStoreProperty.get(), HashRange.all());
}
private List<CompletableFuture<DataUtil.IteratorWithNumBytes>> readBroadcast() {
final int numSrcTasks = this.getSourceParallelism();
- final Optional<DataStoreProperty.Value> dataStoreProperty
- = runtimeEdge.getPropertyValue(DataStoreProperty.class);
+ final Optional<InterTaskDataStoreProperty.Value> dataStoreProperty
+ = runtimeEdge.getPropertyValue(InterTaskDataStoreProperty.class);
final List<CompletableFuture<DataUtil.IteratorWithNumBytes>> futures = new ArrayList<>();
for (int srcTaskIdx = 0; srcTaskIdx < numSrcTasks; srcTaskIdx++) {
@@ -116,8 +116,8 @@ public final class InputReader extends DataTransfer {
*/
private List<CompletableFuture<DataUtil.IteratorWithNumBytes>> readDataInRange() {
assert (runtimeEdge instanceof StageEdge);
- final Optional<DataStoreProperty.Value> dataStoreProperty
- = runtimeEdge.getPropertyValue(DataStoreProperty.class);
+ final Optional<InterTaskDataStoreProperty.Value> dataStoreProperty
+ = runtimeEdge.getPropertyValue(InterTaskDataStoreProperty.class);
final KeyRange hashRangeToRead =
((StageEdge) runtimeEdge).getTaskIdxToKeyRange().get(dstTaskIndex);
if (hashRangeToRead == null) {
@@ -170,8 +170,8 @@ public final class InputReader extends DataTransfer {
* @return the parallelism of the source task.
*/
public int getSourceParallelism() {
- if (CommunicationPatternProperty.Value.OneToOne
- .equals(runtimeEdge.getPropertyValue(CommunicationPatternProperty.class).get())) {
+ if (DataCommunicationPatternProperty.Value.OneToOne
+ .equals(runtimeEdge.getPropertyValue(DataCommunicationPatternProperty.class).get())) {
return 1;
} else {
final Integer numSrcTasks = srcVertex.getPropertyValue(ParallelismProperty.class).get();
diff --git a/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/datatransfer/OutputWriter.java b/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/datatransfer/OutputWriter.java
index 930a4b4..1744bb2 100644
--- a/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/datatransfer/OutputWriter.java
+++ b/runtime/executor/src/main/java/edu/snu/nemo/runtime/executor/datatransfer/OutputWriter.java
@@ -36,7 +36,7 @@ public final class OutputWriter extends DataTransfer implements AutoCloseable {
private final RuntimeEdge<?> runtimeEdge;
private final String srcVertexId;
private final IRVertex dstIrVertex;
- private final DataStoreProperty.Value blockStoreValue;
+ private final InterTaskDataStoreProperty.Value blockStoreValue;
private final BlockManagerWorker blockManagerWorker;
private final boolean nonDummyBlock;
private final Block blockToWrite;
@@ -65,7 +65,7 @@ public final class OutputWriter extends DataTransfer implements AutoCloseable {
this.srcVertexId = srcRuntimeVertexId;
this.dstIrVertex = dstIrVertex;
this.blockManagerWorker = blockManagerWorker;
- this.blockStoreValue = runtimeEdge.getPropertyValue(DataStoreProperty.class).get();
+ this.blockStoreValue = runtimeEdge.getPropertyValue(InterTaskDataStoreProperty.class).get();
// Setup partitioner
final int dstParallelism = getDstParallelism();
@@ -112,8 +112,8 @@ public final class OutputWriter extends DataTransfer implements AutoCloseable {
*/
public void close() {
// Commit block.
- final DataPersistenceProperty.Value persistence =
- runtimeEdge.getPropertyValue(DataPersistenceProperty.class).get();
+ final UsedDataHandlingProperty.Value usedDataHandling =
+ runtimeEdge.getPropertyValue(UsedDataHandlingProperty.class).get();
final Optional<DuplicateEdgeGroupPropertyValue> duplicateDataProperty =
runtimeEdge.getPropertyValue(DuplicateEdgeGroupProperty.class);
final int multiplier = duplicateDataProperty.isPresent() ? duplicateDataProperty.get().getGroupSize() : 1;
@@ -129,11 +129,11 @@ public final class OutputWriter extends DataTransfer implements AutoCloseable {
}
this.writtenBytes = blockSizeTotal;
blockManagerWorker.writeBlock(blockToWrite, blockStoreValue, isDataSizeMetricCollectionEdge,
- partitionSizeMap.get(), srcVertexId, getDstParallelism() * multiplier, persistence);
+ partitionSizeMap.get(), srcVertexId, getDstParallelism() * multiplier, usedDataHandling);
} else {
this.writtenBytes = -1; // no written bytes info.
blockManagerWorker.writeBlock(blockToWrite, blockStoreValue, isDataSizeMetricCollectionEdge,
- Collections.emptyMap(), srcVertexId, getDstParallelism() * multiplier, persistence);
+ Collections.emptyMap(), srcVertexId, getDstParallelism() * multiplier, usedDataHandling);
}
}
@@ -154,8 +154,8 @@ public final class OutputWriter extends DataTransfer implements AutoCloseable {
* @return the parallelism of the destination task.
*/
private int getDstParallelism() {
- return CommunicationPatternProperty.Value.OneToOne.equals(
- runtimeEdge.getPropertyValue(CommunicationPatternProperty.class).get())
+ return DataCommunicationPatternProperty.Value.OneToOne.equals(
+ runtimeEdge.getPropertyValue(DataCommunicationPatternProperty.class).get())
? 1 : dstIrVertex.getPropertyValue(ParallelismProperty.class).get();
}
}
diff --git a/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/datatransfer/DataTransferTest.java b/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/datatransfer/DataTransferTest.java
index b46f7ba..66bc001 100644
--- a/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/datatransfer/DataTransferTest.java
+++ b/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/datatransfer/DataTransferTest.java
@@ -44,6 +44,9 @@ import edu.snu.nemo.runtime.executor.data.BlockManagerWorker;
import edu.snu.nemo.runtime.executor.data.SerializerManager;
import edu.snu.nemo.runtime.master.*;
import edu.snu.nemo.runtime.master.eventhandler.UpdatePhysicalPlanEventHandler;
+import edu.snu.nemo.runtime.master.resource.ContainerManager;
+import edu.snu.nemo.runtime.master.scheduler.ExecutorRegistry;
+import edu.snu.nemo.runtime.master.scheduler.*;
import org.apache.commons.io.FileUtils;
import org.apache.reef.driver.evaluator.EvaluatorRequestor;
import org.apache.reef.io.network.naming.NameResolverConfiguration;
@@ -87,14 +90,14 @@ import static org.mockito.Mockito.mock;
SourceVertex.class, ClientRPC.class, MetricManagerMaster.class})
public final class DataTransferTest {
private static final String EXECUTOR_ID_PREFIX = "Executor";
- private static final DataStoreProperty.Value MEMORY_STORE =
- DataStoreProperty.Value.MemoryStore;
- private static final DataStoreProperty.Value SER_MEMORY_STORE =
- DataStoreProperty.Value.SerializedMemoryStore;
- private static final DataStoreProperty.Value LOCAL_FILE_STORE =
- DataStoreProperty.Value.LocalFileStore;
- private static final DataStoreProperty.Value REMOTE_FILE_STORE =
- DataStoreProperty.Value.GlusterFileStore;
+ private static final InterTaskDataStoreProperty.Value MEMORY_STORE =
+ InterTaskDataStoreProperty.Value.MemoryStore;
+ private static final InterTaskDataStoreProperty.Value SER_MEMORY_STORE =
+ InterTaskDataStoreProperty.Value.SerializedMemoryStore;
+ private static final InterTaskDataStoreProperty.Value LOCAL_FILE_STORE =
+ InterTaskDataStoreProperty.Value.LocalFileStore;
+ private static final InterTaskDataStoreProperty.Value REMOTE_FILE_STORE =
+ InterTaskDataStoreProperty.Value.GlusterFileStore;
private static final String TMP_LOCAL_FILE_DIRECTORY = "./tmpLocalFiles";
private static final String TMP_REMOTE_FILE_DIRECTORY = "./tmpRemoteFiles";
private static final int PARALLELISM_TEN = 10;
@@ -209,82 +212,82 @@ public final class DataTransferTest {
@Test
public void testWriteAndRead() throws Exception {
// test OneToOne same worker
- writeAndRead(worker1, worker1, CommunicationPatternProperty.Value.OneToOne, MEMORY_STORE);
+ writeAndRead(worker1, worker1, DataCommunicationPatternProperty.Value.OneToOne, MEMORY_STORE);
// test OneToOne different worker
- writeAndRead(worker1, worker2, CommunicationPatternProperty.Value.OneToOne, MEMORY_STORE);
+ writeAndRead(worker1, worker2, DataCommunicationPatternProperty.Value.OneToOne, MEMORY_STORE);
// test OneToMany same worker
- writeAndRead(worker1, worker1, CommunicationPatternProperty.Value.BroadCast, MEMORY_STORE);
+ writeAndRead(worker1, worker1, DataCommunicationPatternProperty.Value.BroadCast, MEMORY_STORE);
// test OneToMany different worker
- writeAndRead(worker1, worker2, CommunicationPatternProperty.Value.BroadCast, MEMORY_STORE);
+ writeAndRead(worker1, worker2, DataCommunicationPatternProperty.Value.BroadCast, MEMORY_STORE);
// test ManyToMany same worker
- writeAndRead(worker1, worker1, CommunicationPatternProperty.Value.Shuffle, MEMORY_STORE);
+ writeAndRead(worker1, worker1, DataCommunicationPatternProperty.Value.Shuffle, MEMORY_STORE);
// test ManyToMany different worker
- writeAndRead(worker1, worker2, CommunicationPatternProperty.Value.Shuffle, MEMORY_STORE);
+ writeAndRead(worker1, worker2, DataCommunicationPatternProperty.Value.Shuffle, MEMORY_STORE);
// test ManyToMany same worker
- writeAndRead(worker1, worker1, CommunicationPatternProperty.Value.Shuffle, SER_MEMORY_STORE);
+ writeAndRead(worker1, worker1, DataCommunicationPatternProperty.Value.Shuffle, SER_MEMORY_STORE);
// test ManyToMany different worker
- writeAndRead(worker1, worker2, CommunicationPatternProperty.Value.Shuffle, SER_MEMORY_STORE);
+ writeAndRead(worker1, worker2, DataCommunicationPatternProperty.Value.Shuffle, SER_MEMORY_STORE);
// test ManyToMany same worker (local file)
- writeAndRead(worker1, worker1, CommunicationPatternProperty.Value.Shuffle, LOCAL_FILE_STORE);
+ writeAndRead(worker1, worker1, DataCommunicationPatternProperty.Value.Shuffle, LOCAL_FILE_STORE);
// test ManyToMany different worker (local file)
- writeAndRead(worker1, worker2, CommunicationPatternProperty.Value.Shuffle, LOCAL_FILE_STORE);
+ writeAndRead(worker1, worker2, DataCommunicationPatternProperty.Value.Shuffle, LOCAL_FILE_STORE);
// test ManyToMany same worker (remote file)
- writeAndRead(worker1, worker1, CommunicationPatternProperty.Value.Shuffle, REMOTE_FILE_STORE);
+ writeAndRead(worker1, worker1, DataCommunicationPatternProperty.Value.Shuffle, REMOTE_FILE_STORE);
// test ManyToMany different worker (remote file)
- writeAndRead(worker1, worker2, CommunicationPatternProperty.Value.Shuffle, REMOTE_FILE_STORE);
+ writeAndRead(worker1, worker2, DataCommunicationPatternProperty.Value.Shuffle, REMOTE_FILE_STORE);
// test OneToOne same worker with duplicate data
- writeAndReadWithDuplicateData(worker1, worker1, CommunicationPatternProperty.Value.OneToOne, MEMORY_STORE);
+ writeAndReadWithDuplicateData(worker1, worker1, DataCommunicationPatternProperty.Value.OneToOne, MEMORY_STORE);
// test OneToOne different worker with duplicate data
- writeAndReadWithDuplicateData(worker1, worker2, CommunicationPatternProperty.Value.OneToOne, MEMORY_STORE);
+ writeAndReadWithDuplicateData(worker1, worker2, DataCommunicationPatternProperty.Value.OneToOne, MEMORY_STORE);
// test OneToMany same worker with duplicate data
- writeAndReadWithDuplicateData(worker1, worker1, CommunicationPatternProperty.Value.BroadCast, MEMORY_STORE);
+ writeAndReadWithDuplicateData(worker1, worker1, DataCommunicationPatternProperty.Value.BroadCast, MEMORY_STORE);
// test OneToMany different worker with duplicate data
- writeAndReadWithDuplicateData(worker1, worker2, CommunicationPatternProperty.Value.BroadCast, MEMORY_STORE);
+ writeAndReadWithDuplicateData(worker1, worker2, DataCommunicationPatternProperty.Value.BroadCast, MEMORY_STORE);
// test ManyToMany same worker with duplicate data
- writeAndReadWithDuplicateData(worker1, worker1, CommunicationPatternProperty.Value.Shuffle, MEMORY_STORE);
+ writeAndReadWithDuplicateData(worker1, worker1, DataCommunicationPatternProperty.Value.Shuffle, MEMORY_STORE);
// test ManyToMany different worker with duplicate data
- writeAndReadWithDuplicateData(worker1, worker2, CommunicationPatternProperty.Value.Shuffle, MEMORY_STORE);
+ writeAndReadWithDuplicateData(worker1, worker2, DataCommunicationPatternProperty.Value.Shuffle, MEMORY_STORE);
// test ManyToMany same worker with duplicate data
- writeAndReadWithDuplicateData(worker1, worker1, CommunicationPatternProperty.Value.Shuffle, SER_MEMORY_STORE);
+ writeAndReadWithDuplicateData(worker1, worker1, DataCommunicationPatternProperty.Value.Shuffle, SER_MEMORY_STORE);
// test ManyToMany different worker with duplicate data
- writeAndReadWithDuplicateData(worker1, worker2, CommunicationPatternProperty.Value.Shuffle, SER_MEMORY_STORE);
+ writeAndReadWithDuplicateData(worker1, worker2, DataCommunicationPatternProperty.Value.Shuffle, SER_MEMORY_STORE);
// test ManyToMany same worker (local file) with duplicate data
- writeAndReadWithDuplicateData(worker1, worker1, CommunicationPatternProperty.Value.Shuffle, LOCAL_FILE_STORE);
+ writeAndReadWithDuplicateData(worker1, worker1, DataCommunicationPatternProperty.Value.Shuffle, LOCAL_FILE_STORE);
// test ManyToMany different worker (local file) with duplicate data
- writeAndReadWithDuplicateData(worker1, worker2, CommunicationPatternProperty.Value.Shuffle, LOCAL_FILE_STORE);
+ writeAndReadWithDuplicateData(worker1, worker2, DataCommunicationPatternProperty.Value.Shuffle, LOCAL_FILE_STORE);
// test ManyToMany same worker (remote file) with duplicate data
- writeAndReadWithDuplicateData(worker1, worker1, CommunicationPatternProperty.Value.Shuffle, REMOTE_FILE_STORE);
+ writeAndReadWithDuplicateData(worker1, worker1, DataCommunicationPatternProperty.Value.Shuffle, REMOTE_FILE_STORE);
// test ManyToMany different worker (remote file) with duplicate data
- writeAndReadWithDuplicateData(worker1, worker2, CommunicationPatternProperty.Value.Shuffle, REMOTE_FILE_STORE);
+ writeAndReadWithDuplicateData(worker1, worker2, DataCommunicationPatternProperty.Value.Shuffle, REMOTE_FILE_STORE);
}
private void writeAndRead(final BlockManagerWorker sender,
final BlockManagerWorker receiver,
- final CommunicationPatternProperty.Value commPattern,
- final DataStoreProperty.Value store) throws RuntimeException {
+ final DataCommunicationPatternProperty.Value commPattern,
+ final InterTaskDataStoreProperty.Value store) throws RuntimeException {
final int testIndex = TEST_INDEX.getAndIncrement();
final String edgeId = String.format(EDGE_PREFIX_TEMPLATE, testIndex);
final Pair<IRVertex, IRVertex> verticesPair = setupVertices(edgeId, sender, receiver);
@@ -294,10 +297,10 @@ public final class DataTransferTest {
// Edge setup
final IREdge dummyIREdge = new IREdge(commPattern, srcVertex, dstVertex);
dummyIREdge.setProperty(KeyExtractorProperty.of((element -> element)));
- dummyIREdge.setProperty(CommunicationPatternProperty.of(commPattern));
+ dummyIREdge.setProperty(DataCommunicationPatternProperty.of(commPattern));
dummyIREdge.setProperty(PartitionerProperty.of(PartitionerProperty.Value.HashPartitioner));
- dummyIREdge.setProperty(DataStoreProperty.of(store));
- dummyIREdge.setProperty(DataPersistenceProperty.of(DataPersistenceProperty.Value.Keep));
+ dummyIREdge.setProperty(InterTaskDataStoreProperty.of(store));
+ dummyIREdge.setProperty(UsedDataHandlingProperty.of(UsedDataHandlingProperty.Value.Keep));
dummyIREdge.setProperty(EncoderProperty.of(ENCODER_FACTORY));
dummyIREdge.setProperty(DecoderProperty.of(DECODER_FACTORY));
final ExecutionPropertyMap edgeProperties = dummyIREdge.getExecutionProperties();
@@ -334,7 +337,7 @@ public final class DataTransferTest {
final InputReader reader =
new InputReader(dstTaskIndex, srcVertex, dummyEdge, receiver);
- if (CommunicationPatternProperty.Value.OneToOne.equals(commPattern)) {
+ if (DataCommunicationPatternProperty.Value.OneToOne.equals(commPattern)) {
assertEquals(1, reader.getSourceParallelism());
} else {
assertEquals(PARALLELISM_TEN, reader.getSourceParallelism());
@@ -352,7 +355,7 @@ public final class DataTransferTest {
// Compare (should be the same)
final List flattenedWrittenData = flatten(dataWrittenList);
final List flattenedReadData = flatten(dataReadList);
- if (CommunicationPatternProperty.Value.BroadCast.equals(commPattern)) {
+ if (DataCommunicationPatternProperty.Value.BroadCast.equals(commPattern)) {
final List broadcastedWrittenData = new ArrayList<>();
IntStream.range(0, PARALLELISM_TEN).forEach(i -> broadcastedWrittenData.addAll(flattenedWrittenData));
assertEquals(broadcastedWrittenData.size(), flattenedReadData.size());
@@ -365,8 +368,8 @@ public final class DataTransferTest {
private void writeAndReadWithDuplicateData(final BlockManagerWorker sender,
final BlockManagerWorker receiver,
- final CommunicationPatternProperty.Value commPattern,
- final DataStoreProperty.Value store) throws RuntimeException {
+ final DataCommunicationPatternProperty.Value commPattern,
+ final InterTaskDataStoreProperty.Value store) throws RuntimeException {
final int testIndex = TEST_INDEX.getAndIncrement();
final int testIndex2 = TEST_INDEX.getAndIncrement();
final String edgeId = String.format(EDGE_PREFIX_TEMPLATE, testIndex);
@@ -380,15 +383,15 @@ public final class DataTransferTest {
dummyIREdge.setProperty(EncoderProperty.of(ENCODER_FACTORY));
dummyIREdge.setProperty(DecoderProperty.of(DECODER_FACTORY));
dummyIREdge.setProperty(KeyExtractorProperty.of((element -> element)));
- dummyIREdge.setProperty(CommunicationPatternProperty.of(commPattern));
+ dummyIREdge.setProperty(DataCommunicationPatternProperty.of(commPattern));
dummyIREdge.setProperty(PartitionerProperty.of(PartitionerProperty.Value.HashPartitioner));
dummyIREdge.setProperty(DuplicateEdgeGroupProperty.of(new DuplicateEdgeGroupPropertyValue("dummy")));
final Optional<DuplicateEdgeGroupPropertyValue> duplicateDataProperty
= dummyIREdge.getPropertyValue(DuplicateEdgeGroupProperty.class);
duplicateDataProperty.get().setRepresentativeEdgeId(edgeId);
duplicateDataProperty.get().setGroupSize(2);
- dummyIREdge.setProperty(DataStoreProperty.of(store));
- dummyIREdge.setProperty(DataPersistenceProperty.of(DataPersistenceProperty.Value.Keep));
+ dummyIREdge.setProperty(InterTaskDataStoreProperty.of(store));
+ dummyIREdge.setProperty(UsedDataHandlingProperty.of(UsedDataHandlingProperty.Value.Keep));
final RuntimeEdge dummyEdge, dummyEdge2;
final ExecutionPropertyMap edgeProperties = dummyIREdge.getExecutionProperties();
@@ -436,13 +439,13 @@ public final class DataTransferTest {
final InputReader reader2 =
new InputReader(dstTaskIndex, srcVertex, dummyEdge2, receiver);
- if (CommunicationPatternProperty.Value.OneToOne.equals(commPattern)) {
+ if (DataCommunicationPatternProperty.Value.OneToOne.equals(commPattern)) {
assertEquals(1, reader.getSourceParallelism());
} else {
assertEquals(PARALLELISM_TEN, reader.getSourceParallelism());
}
- if (CommunicationPatternProperty.Value.OneToOne.equals(commPattern)) {
+ if (DataCommunicationPatternProperty.Value.OneToOne.equals(commPattern)) {
assertEquals(1, reader2.getSourceParallelism());
} else {
assertEquals(PARALLELISM_TEN, reader2.getSourceParallelism());
@@ -470,7 +473,7 @@ public final class DataTransferTest {
final List flattenedWrittenData2 = flatten(dataWrittenList);
final List flattenedReadData = flatten(dataReadList);
final List flattenedReadData2 = flatten(dataReadList2);
- if (CommunicationPatternProperty.Value.BroadCast.equals(commPattern)) {
+ if (DataCommunicationPatternProperty.Value.BroadCast.equals(commPattern)) {
final List broadcastedWrittenData = new ArrayList<>();
final List broadcastedWrittenData2 = new ArrayList<>();
IntStream.range(0, PARALLELISM_TEN).forEach(i -> broadcastedWrittenData.addAll(flattenedWrittenData));
diff --git a/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/task/TaskExecutorTest.java b/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/task/TaskExecutorTest.java
index 51e6c83..f9c4663 100644
--- a/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/task/TaskExecutorTest.java
+++ b/runtime/executor/src/test/java/edu/snu/nemo/runtime/executor/task/TaskExecutorTest.java
@@ -20,7 +20,7 @@ import edu.snu.nemo.common.ir.OutputCollector;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.Readable;
-import edu.snu.nemo.common.ir.edge.executionproperty.DataStoreProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.InterTaskDataStoreProperty;
import edu.snu.nemo.common.ir.executionproperty.VertexExecutionProperty;
import edu.snu.nemo.common.ir.vertex.InMemorySourceVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
@@ -328,7 +328,7 @@ public final class TaskExecutorTest {
final boolean isSideInput) {
final String runtimeIREdgeId = "Runtime edge between operator tasks";
ExecutionPropertyMap edgeProperties = new ExecutionPropertyMap(runtimeIREdgeId);
- edgeProperties.put(DataStoreProperty.of(DataStoreProperty.Value.MemoryStore));
+ edgeProperties.put(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.MemoryStore));
return new RuntimeEdge<>(runtimeIREdgeId, edgeProperties, src, dst, isSideInput);
}
@@ -338,7 +338,7 @@ public final class TaskExecutorTest {
final boolean isSideInput,
final String runtimeIREdgeId) {
ExecutionPropertyMap edgeProperties = new ExecutionPropertyMap(runtimeIREdgeId);
- edgeProperties.put(DataStoreProperty.of(DataStoreProperty.Value.MemoryStore));
+ edgeProperties.put(InterTaskDataStoreProperty.of(InterTaskDataStoreProperty.Value.MemoryStore));
return new RuntimeEdge<>(runtimeIREdgeId, edgeProperties, src, dst, isSideInput);
}
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/RuntimeMaster.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/RuntimeMaster.java
index 0aef6a6..91c385c 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/RuntimeMaster.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/RuntimeMaster.java
@@ -406,7 +406,7 @@ public final class RuntimeMaster {
metricCollectionBarrierVertex.setMetricData(aggregatedMetricData);
});
} else {
- throw new RuntimeException("Something wrong happened at SkewCompositePass.");
+ throw new RuntimeException("Something wrong happened at DataSkewCompositePass.");
}
}
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/resource/ExecutorRepresenter.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/resource/ExecutorRepresenter.java
index d0e21d8..6a0a3ee 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/resource/ExecutorRepresenter.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/resource/ExecutorRepresenter.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.runtime.master.resource;
import com.google.protobuf.ByteString;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorSlotComplianceProperty;
import edu.snu.nemo.runtime.common.RuntimeIdGenerator;
import edu.snu.nemo.runtime.common.comm.ControlMessage;
import edu.snu.nemo.runtime.common.message.MessageEnvironment;
@@ -103,7 +103,7 @@ public final class ExecutorRepresenter {
* @param task
*/
public void onTaskScheduled(final Task task) {
- (task.getPropertyValue(ResourceSlotProperty.class).orElse(true)
+ (task.getPropertyValue(ExecutorSlotComplianceProperty.class).orElse(true)
? runningComplyingTasks : runningNonComplyingTasks).put(task.getTaskId(), task);
runningTaskToAttempt.put(task, task.getAttemptIdx());
failedTasks.remove(task);
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraint.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraint.java
index 9e79ec1..55c3824 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraint.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraint.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.runtime.master.scheduler;
import edu.snu.nemo.common.ir.executionproperty.AssociatedProperty;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
import edu.snu.nemo.runtime.common.plan.Task;
import edu.snu.nemo.runtime.master.resource.ExecutorRepresenter;
@@ -25,7 +25,7 @@ import javax.inject.Inject;
/**
* This policy find executors which has corresponding container type.
*/
-@AssociatedProperty(ResourcePriorityProperty.class)
+@AssociatedProperty(ExecutorPlacementProperty.class)
public final class ContainerTypeAwareSchedulingConstraint implements SchedulingConstraint {
@Inject
@@ -34,9 +34,9 @@ public final class ContainerTypeAwareSchedulingConstraint implements SchedulingC
@Override
public boolean testSchedulability(final ExecutorRepresenter executor, final Task task) {
- final String executorPlacementPropertyValue = task.getPropertyValue(ResourcePriorityProperty.class)
- .orElse(ResourcePriorityProperty.NONE);
- return executorPlacementPropertyValue.equals(ResourcePriorityProperty.NONE) ? true
+ final String executorPlacementPropertyValue = task.getPropertyValue(ExecutorPlacementProperty.class)
+ .orElse(ExecutorPlacementProperty.NONE);
+ return executorPlacementPropertyValue.equals(ExecutorPlacementProperty.NONE) ? true
: executor.getContainerType().equals(executorPlacementPropertyValue);
}
}
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/FreeSlotSchedulingConstraint.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/FreeSlotSchedulingConstraint.java
index 1726cdb..c330cd4 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/FreeSlotSchedulingConstraint.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/FreeSlotSchedulingConstraint.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.runtime.master.scheduler;
import edu.snu.nemo.common.ir.executionproperty.AssociatedProperty;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorSlotComplianceProperty;
import edu.snu.nemo.runtime.common.plan.Task;
import edu.snu.nemo.runtime.master.resource.ExecutorRepresenter;
@@ -25,7 +25,7 @@ import javax.inject.Inject;
/**
* This policy finds executor that has free slot for a Task.
*/
-@AssociatedProperty(ResourceSlotProperty.class)
+@AssociatedProperty(ExecutorSlotComplianceProperty.class)
public final class FreeSlotSchedulingConstraint implements SchedulingConstraint {
@Inject
@@ -34,7 +34,7 @@ public final class FreeSlotSchedulingConstraint implements SchedulingConstraint
@Override
public boolean testSchedulability(final ExecutorRepresenter executor, final Task task) {
- if (!task.getPropertyValue(ResourceSlotProperty.class).orElse(false)) {
+ if (!task.getPropertyValue(ExecutorSlotComplianceProperty.class).orElse(false)) {
return true;
}
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/NodeShareSchedulingConstraint.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/NodeShareSchedulingConstraint.java
index ff3986e..6fcbc93 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/NodeShareSchedulingConstraint.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/NodeShareSchedulingConstraint.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.runtime.master.scheduler;
import edu.snu.nemo.common.ir.executionproperty.AssociatedProperty;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceSiteProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.NodeNamesProperty;
import edu.snu.nemo.runtime.common.RuntimeIdGenerator;
import edu.snu.nemo.runtime.common.plan.Task;
import edu.snu.nemo.runtime.master.resource.ExecutorRepresenter;
@@ -25,9 +25,9 @@ import javax.inject.Inject;
import java.util.*;
/**
- * This constraint is to follow {@link ResourceSiteProperty}.
+ * This constraint is to follow {@link NodeNamesProperty}.
*/
-@AssociatedProperty(ResourceSiteProperty.class)
+@AssociatedProperty(NodeNamesProperty.class)
public final class NodeShareSchedulingConstraint implements SchedulingConstraint {
@Inject
@@ -45,13 +45,13 @@ public final class NodeShareSchedulingConstraint implements SchedulingConstraint
return nodeName;
}
}
- throw new IllegalStateException("Detected excessive parallelism which ResourceSiteProperty does not cover");
+ throw new IllegalStateException("Detected excessive parallelism which NodeNamesProperty does not cover");
}
@Override
public boolean testSchedulability(final ExecutorRepresenter executor, final Task task) {
- final Map<String, Integer> propertyValue = task.getPropertyValue(ResourceSiteProperty.class)
- .orElseThrow(() -> new RuntimeException("ResourceSiteProperty expected"));
+ final Map<String, Integer> propertyValue = task.getPropertyValue(NodeNamesProperty.class)
+ .orElseThrow(() -> new RuntimeException("NodeNamesProperty expected"));
if (propertyValue.isEmpty()) {
return true;
}
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SkewnessAwareSchedulingConstraint.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SkewnessAwareSchedulingConstraint.java
index 236453f..6ac14cf 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SkewnessAwareSchedulingConstraint.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SkewnessAwareSchedulingConstraint.java
@@ -17,7 +17,7 @@ package edu.snu.nemo.runtime.master.scheduler;
import com.google.common.annotations.VisibleForTesting;
import edu.snu.nemo.common.ir.executionproperty.AssociatedProperty;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceSkewedDataProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.SkewnessAwareSchedulingProperty;
import edu.snu.nemo.runtime.common.RuntimeIdGenerator;
import edu.snu.nemo.runtime.common.data.HashRange;
import edu.snu.nemo.runtime.common.data.KeyRange;
@@ -34,7 +34,7 @@ import javax.inject.Inject;
*/
@ThreadSafe
@DriverSide
-@AssociatedProperty(ResourceSkewedDataProperty.class)
+@AssociatedProperty(SkewnessAwareSchedulingProperty.class)
public final class SkewnessAwareSchedulingConstraint implements SchedulingConstraint {
@VisibleForTesting
diff --git a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraint.java b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraint.java
index f446362..e0d79a0 100644
--- a/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraint.java
+++ b/runtime/master/src/main/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraint.java
@@ -17,7 +17,7 @@ package edu.snu.nemo.runtime.master.scheduler;
import edu.snu.nemo.common.ir.Readable;
import edu.snu.nemo.common.ir.executionproperty.AssociatedProperty;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.SourceLocationAwareSchedulingProperty;
import edu.snu.nemo.runtime.common.plan.Task;
import edu.snu.nemo.runtime.master.resource.ExecutorRepresenter;
import org.apache.reef.annotations.audience.DriverSide;
@@ -33,7 +33,7 @@ import java.util.*;
*/
@ThreadSafe
@DriverSide
-@AssociatedProperty(ResourceLocalityProperty.class)
+@AssociatedProperty(SourceLocationAwareSchedulingProperty.class)
public final class SourceLocationAwareSchedulingConstraint implements SchedulingConstraint {
@Inject
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/ContainerManagerTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/ContainerManagerTest.java
index ac8a1c7..441dd31 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/ContainerManagerTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/ContainerManagerTest.java
@@ -15,7 +15,7 @@
*/
package edu.snu.nemo.runtime.master;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
import edu.snu.nemo.conf.JobConf;
import edu.snu.nemo.runtime.common.message.MessageEnvironment;
import edu.snu.nemo.runtime.master.resource.ContainerManager;
@@ -35,8 +35,10 @@ import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
+import java.util.Optional;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.IntStream;
import static org.junit.Assert.assertEquals;
import static org.mockito.Matchers.anyString;
@@ -48,11 +50,11 @@ import static org.mockito.Mockito.when;
*/
public final class ContainerManagerTest {
private static final ResourceSpecification RESOURCE_SPEC_A =
- new ResourceSpecification(ResourcePriorityProperty.COMPUTE, 1, 1024);
+ new ResourceSpecification(ExecutorPlacementProperty.COMPUTE, 1, 1024);
private static final ResourceSpecification RESOURCE_SPEC_B =
- new ResourceSpecification(ResourcePriorityProperty.TRANSIENT, 2, 2048);
+ new ResourceSpecification(ExecutorPlacementProperty.TRANSIENT, 2, 2048);
private static final ResourceSpecification RESOURCE_SPEC_C =
- new ResourceSpecification(ResourcePriorityProperty.RESERVED, 3, 3072);
+ new ResourceSpecification(ExecutorPlacementProperty.RESERVED, 3, 3072);
private ContainerManager containerManager;
private AtomicInteger testIdNumber = new AtomicInteger(0);
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/BatchSingleJobSchedulerTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/BatchSingleJobSchedulerTest.java
index cfa6194..935f310 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/BatchSingleJobSchedulerTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/BatchSingleJobSchedulerTest.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.runtime.master.scheduler;
import edu.snu.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
import edu.snu.nemo.conf.JobConf;
import edu.snu.nemo.runtime.common.comm.ControlMessage;
import edu.snu.nemo.runtime.common.message.MessageSender;
@@ -87,7 +87,7 @@ public final class BatchSingleJobSchedulerTest {
final ExecutorService serializationExecutorService = Executors.newSingleThreadExecutor();
final ResourceSpecification computeSpec =
- new ResourceSpecification(ResourcePriorityProperty.COMPUTE, EXECUTOR_CAPACITY, 0);
+ new ResourceSpecification(ExecutorPlacementProperty.COMPUTE, EXECUTOR_CAPACITY, 0);
final Function<String, ExecutorRepresenter> computeSpecExecutorRepresenterGenerator = executorId ->
new ExecutorRepresenter(executorId, computeSpec, mockMsgSender, activeContext, serializationExecutorService,
executorId);
@@ -96,7 +96,7 @@ public final class BatchSingleJobSchedulerTest {
final ExecutorRepresenter a1 = computeSpecExecutorRepresenterGenerator.apply("a1");
final ResourceSpecification storageSpec =
- new ResourceSpecification(ResourcePriorityProperty.TRANSIENT, EXECUTOR_CAPACITY, 0);
+ new ResourceSpecification(ExecutorPlacementProperty.TRANSIENT, EXECUTOR_CAPACITY, 0);
final Function<String, ExecutorRepresenter> storageSpecExecutorRepresenterGenerator = executorId ->
new ExecutorRepresenter(executorId, storageSpec, mockMsgSender, activeContext, serializationExecutorService,
executorId);
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraintTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraintTest.java
index 9e3f87c..574d912 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraintTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/ContainerTypeAwareSchedulingConstraintTest.java
@@ -15,7 +15,7 @@
*/
package edu.snu.nemo.runtime.master.scheduler;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
import edu.snu.nemo.runtime.common.plan.Task;
import edu.snu.nemo.runtime.master.resource.ExecutorRepresenter;
import org.apache.reef.tang.Tang;
@@ -48,13 +48,13 @@ public final class ContainerTypeAwareSchedulingConstraintTest {
public void testContainerTypeAware() throws InjectionException {
final SchedulingConstraint schedulingConstraint = Tang.Factory.getTang().newInjector()
.getInstance(ContainerTypeAwareSchedulingConstraint.class);
- final ExecutorRepresenter a0 = mockExecutorRepresenter(ResourcePriorityProperty.TRANSIENT);
- final ExecutorRepresenter a1 = mockExecutorRepresenter(ResourcePriorityProperty.RESERVED);
- final ExecutorRepresenter a2 = mockExecutorRepresenter(ResourcePriorityProperty.NONE);
+ final ExecutorRepresenter a0 = mockExecutorRepresenter(ExecutorPlacementProperty.TRANSIENT);
+ final ExecutorRepresenter a1 = mockExecutorRepresenter(ExecutorPlacementProperty.RESERVED);
+ final ExecutorRepresenter a2 = mockExecutorRepresenter(ExecutorPlacementProperty.NONE);
final Task task1 = mock(Task.class);
- when(task1.getPropertyValue(ResourcePriorityProperty.class))
- .thenReturn(Optional.of(ResourcePriorityProperty.RESERVED));
+ when(task1.getPropertyValue(ExecutorPlacementProperty.class))
+ .thenReturn(Optional.of(ExecutorPlacementProperty.RESERVED));
final Set<ExecutorRepresenter> executorRepresenterList1 = new HashSet<>(Arrays.asList(a0, a1, a2));
@@ -66,8 +66,8 @@ public final class ContainerTypeAwareSchedulingConstraintTest {
assertEquals(expectedExecutors1, candidateExecutors1);
final Task task2 = mock(Task.class);
- when(task2.getPropertyValue(ResourcePriorityProperty.class))
- .thenReturn(Optional.of(ResourcePriorityProperty.NONE));
+ when(task2.getPropertyValue(ExecutorPlacementProperty.class))
+ .thenReturn(Optional.of(ExecutorPlacementProperty.NONE));
final Set<ExecutorRepresenter> executorRepresenterList2 = new HashSet<>(Arrays.asList(a0, a1, a2));
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/FreeSlotSchedulingConstraintTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/FreeSlotSchedulingConstraintTest.java
index 215a93c..871be92 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/FreeSlotSchedulingConstraintTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/FreeSlotSchedulingConstraintTest.java
@@ -15,7 +15,7 @@
*/
package edu.snu.nemo.runtime.master.scheduler;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorSlotComplianceProperty;
import edu.snu.nemo.runtime.common.plan.Task;
import edu.snu.nemo.runtime.master.resource.ExecutorRepresenter;
import org.apache.reef.tang.Tang;
@@ -70,7 +70,7 @@ public final class FreeSlotSchedulingConstraintTest {
public void testFreeSlot() {
final Task task = mock(Task.class);
- when(task.getPropertyValue(ResourceSlotProperty.class)).thenReturn(Optional.of(true));
+ when(task.getPropertyValue(ExecutorSlotComplianceProperty.class)).thenReturn(Optional.of(true));
final Set<ExecutorRepresenter> executorRepresenterList = new HashSet<>(Arrays.asList(a0, a1));
@@ -89,7 +89,7 @@ public final class FreeSlotSchedulingConstraintTest {
public void testIgnoringSlot() {
final Task task = mock(Task.class);
- when(task.getPropertyValue(ResourceSlotProperty.class)).thenReturn(Optional.of(false));
+ when(task.getPropertyValue(ExecutorSlotComplianceProperty.class)).thenReturn(Optional.of(false));
final Set<ExecutorRepresenter> executorRepresenterList = new HashSet<>(Arrays.asList(a0, a1));
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/SchedulingConstraintnRegistryTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/SchedulingConstraintnRegistryTest.java
index f7fb982..443ffca 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/SchedulingConstraintnRegistryTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/SchedulingConstraintnRegistryTest.java
@@ -16,9 +16,9 @@
package edu.snu.nemo.runtime.master.scheduler;
import edu.snu.nemo.common.ir.executionproperty.VertexExecutionProperty;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceSlotProperty;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorSlotComplianceProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.SourceLocationAwareSchedulingProperty;
import org.apache.reef.tang.Tang;
import org.apache.reef.tang.exceptions.InjectionException;
import org.junit.Test;
@@ -33,11 +33,11 @@ public final class SchedulingConstraintnRegistryTest {
public void testSchedulingConstraintRegistry() throws InjectionException {
final SchedulingConstraintRegistry registry = Tang.Factory.getTang().newInjector()
.getInstance(SchedulingConstraintRegistry.class);
- assertEquals(FreeSlotSchedulingConstraint.class, getConstraintOf(ResourceSlotProperty.class, registry));
+ assertEquals(FreeSlotSchedulingConstraint.class, getConstraintOf(ExecutorSlotComplianceProperty.class, registry));
assertEquals(ContainerTypeAwareSchedulingConstraint.class,
- getConstraintOf(ResourcePriorityProperty.class, registry));
+ getConstraintOf(ExecutorPlacementProperty.class, registry));
assertEquals(SourceLocationAwareSchedulingConstraint.class,
- getConstraintOf(ResourceLocalityProperty.class, registry));
+ getConstraintOf(SourceLocationAwareSchedulingProperty.class, registry));
}
private static Class<? extends SchedulingConstraint> getConstraintOf(
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraintTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraintTest.java
index 1beb8f9..81bd65d 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraintTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/SourceLocationAwareSchedulingConstraintTest.java
@@ -15,7 +15,7 @@
*/
package edu.snu.nemo.runtime.master.scheduler;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourceLocalityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.SourceLocationAwareSchedulingProperty;
import edu.snu.nemo.runtime.common.plan.Task;
import edu.snu.nemo.common.ir.Readable;
import edu.snu.nemo.runtime.master.resource.ExecutorRepresenter;
@@ -111,7 +111,7 @@ public final class SourceLocationAwareSchedulingConstraintTest {
readable));
when(mockInstance.getTaskId()).thenReturn(String.format("T-%d", taskIndex.getAndIncrement()));
when(mockInstance.getIrVertexIdToReadable()).thenReturn(readableMap);
- when(mockInstance.getPropertyValue(ResourceLocalityProperty.class)).thenReturn(Optional.of(true));
+ when(mockInstance.getPropertyValue(SourceLocationAwareSchedulingProperty.class)).thenReturn(Optional.of(true));
return mockInstance;
}
diff --git a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/TaskRetryTest.java b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/TaskRetryTest.java
index cd54432..d9b9b30 100644
--- a/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/TaskRetryTest.java
+++ b/runtime/master/src/test/java/edu/snu/nemo/runtime/master/scheduler/TaskRetryTest.java
@@ -16,7 +16,7 @@
package edu.snu.nemo.runtime.master.scheduler;
import edu.snu.nemo.common.eventhandler.PubSubEventHandlerWrapper;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
import edu.snu.nemo.runtime.common.comm.ControlMessage;
import edu.snu.nemo.runtime.common.message.MessageSender;
import edu.snu.nemo.runtime.common.plan.PhysicalPlan;
@@ -147,7 +147,7 @@ public final class TaskRetryTest {
final ActiveContext activeContext = mock(ActiveContext.class);
Mockito.doThrow(new RuntimeException()).when(activeContext).close();
final ExecutorService serExecutorService = Executors.newSingleThreadExecutor();
- final ResourceSpecification computeSpec = new ResourceSpecification(ResourcePriorityProperty.COMPUTE, 2, 0);
+ final ResourceSpecification computeSpec = new ResourceSpecification(ExecutorPlacementProperty.COMPUTE, 2, 0);
final ExecutorRepresenter executor = new ExecutorRepresenter("EXECUTOR" + ID_OFFSET.getAndIncrement(),
computeSpec, mockMsgSender, activeContext, serExecutorService, "NODE" + ID_OFFSET.getAndIncrement());
scheduler.onExecutorAdded(executor);
diff --git a/runtime/test/src/main/java/edu/snu/nemo/runtime/common/plan/TestPlanGenerator.java b/runtime/test/src/main/java/edu/snu/nemo/runtime/common/plan/TestPlanGenerator.java
index 93c7ad5..04c2b06 100644
--- a/runtime/test/src/main/java/edu/snu/nemo/runtime/common/plan/TestPlanGenerator.java
+++ b/runtime/test/src/main/java/edu/snu/nemo/runtime/common/plan/TestPlanGenerator.java
@@ -18,10 +18,10 @@ package edu.snu.nemo.runtime.common.plan;
import edu.snu.nemo.common.dag.DAG;
import edu.snu.nemo.common.dag.DAGBuilder;
import edu.snu.nemo.common.ir.edge.IREdge;
-import edu.snu.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
+import edu.snu.nemo.common.ir.edge.executionproperty.DataCommunicationPatternProperty;
import edu.snu.nemo.common.ir.vertex.IRVertex;
import edu.snu.nemo.common.ir.vertex.OperatorVertex;
-import edu.snu.nemo.common.ir.vertex.executionproperty.ResourcePriorityProperty;
+import edu.snu.nemo.common.ir.vertex.executionproperty.ExecutorPlacementProperty;
import edu.snu.nemo.common.ir.vertex.executionproperty.ParallelismProperty;
import edu.snu.nemo.common.ir.vertex.transform.Transform;
import edu.snu.nemo.common.test.EmptyComponents;
@@ -108,39 +108,39 @@ public final class TestPlanGenerator {
final Transform t = new EmptyComponents.EmptyTransform("empty");
final IRVertex v1 = new OperatorVertex(t);
v1.setProperty(ParallelismProperty.of(3));
- v1.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v1.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
dagBuilder.addVertex(v1);
final IRVertex v2 = new OperatorVertex(t);
v2.setProperty(ParallelismProperty.of(2));
- v2.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v2.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
dagBuilder.addVertex(v2);
final IRVertex v3 = new OperatorVertex(t);
v3.setProperty(ParallelismProperty.of(3));
- v3.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v3.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
dagBuilder.addVertex(v3);
final IRVertex v4 = new OperatorVertex(t);
v4.setProperty(ParallelismProperty.of(2));
- v4.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v4.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
dagBuilder.addVertex(v4);
final IRVertex v5 = new OperatorVertex(t);
v5.setProperty(ParallelismProperty.of(2));
- v5.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v5.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
dagBuilder.addVertex(v5);
- final IREdge e1 = new IREdge(CommunicationPatternProperty.Value.Shuffle, v1, v2);
+ final IREdge e1 = new IREdge(DataCommunicationPatternProperty.Value.Shuffle, v1, v2);
dagBuilder.connectVertices(e1);
- final IREdge e2 = new IREdge(CommunicationPatternProperty.Value.Shuffle, v3, v2);
+ final IREdge e2 = new IREdge(DataCommunicationPatternProperty.Value.Shuffle, v3, v2);
dagBuilder.connectVertices(e2);
- final IREdge e3 = new IREdge(CommunicationPatternProperty.Value.Shuffle, v2, v4);
+ final IREdge e3 = new IREdge(DataCommunicationPatternProperty.Value.Shuffle, v2, v4);
dagBuilder.connectVertices(e3);
- final IREdge e4 = new IREdge(CommunicationPatternProperty.Value.OneToOne, v4, v5);
+ final IREdge e4 = new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v4, v5);
dagBuilder.connectVertices(e4);
return dagBuilder.buildWithoutSourceSinkCheck();
@@ -156,31 +156,31 @@ public final class TestPlanGenerator {
final Transform t = new EmptyComponents.EmptyTransform("empty");
final IRVertex v1 = new OperatorVertex(t);
v1.setProperty(ParallelismProperty.of(3));
- v1.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v1.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
dagBuilder.addVertex(v1);
final IRVertex v2 = new OperatorVertex(t);
v2.setProperty(ParallelismProperty.of(2));
if (sameContainerType) {
- v2.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v2.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
} else {
- v2.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.TRANSIENT));
+ v2.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.TRANSIENT));
}
dagBuilder.addVertex(v2);
final IRVertex v3 = new OperatorVertex(t);
v3.setProperty(ParallelismProperty.of(2));
if (sameContainerType) {
- v3.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.COMPUTE));
+ v3.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.COMPUTE));
} else {
- v3.setProperty(ResourcePriorityProperty.of(ResourcePriorityProperty.TRANSIENT));
+ v3.setProperty(ExecutorPlacementProperty.of(ExecutorPlacementProperty.TRANSIENT));
}
dagBuilder.addVertex(v3);
- final IREdge e1 = new IREdge(CommunicationPatternProperty.Value.Shuffle, v1, v2);
+ final IREdge e1 = new IREdge(DataCommunicationPatternProperty.Value.Shuffle, v1, v2);
dagBuilder.connectVertices(e1);
- final IREdge e2 = new IREdge(CommunicationPatternProperty.Value.OneToOne, v2, v3);
+ final IREdge e2 = new IREdge(DataCommunicationPatternProperty.Value.OneToOne, v2, v3);
dagBuilder.connectVertices(e2);
return dagBuilder.buildWithoutSourceSinkCheck();