|
16 | 16 | * limitations under the License. |
17 | 17 | */ |
18 | 18 |
|
19 | | -package org.apache.flink.ml.common.sharedstorage; |
| 19 | +package org.apache.flink.ml.common.sharedobjects; |
20 | 20 |
|
21 | 21 | import org.apache.flink.annotation.Experimental; |
| 22 | +import org.apache.flink.api.dag.Transformation; |
22 | 23 | import org.apache.flink.streaming.api.datastream.DataStream; |
23 | 24 |
|
24 | 25 | import java.io.Serializable; |
25 | 26 | import java.util.List; |
26 | 27 | import java.util.Map; |
27 | 28 |
|
28 | 29 | /** |
29 | | - * The builder of the subgraph that will be executed with a common shared storage. Users can only |
| 30 | + * The builder of the subgraph that will be executed with a common shared objects. Users can only |
30 | 31 | * create data streams from {@code inputs}. Users can not refer to data streams outside, and can not |
31 | 32 | * add sources/sinks. |
32 | 33 | * |
33 | | - * <p>The shared storage body requires all streams accessing the shared storage, i.e., {@link |
34 | | - * SharedStorageBodyResult#accessors} have same parallelism and can be co-located. |
| 34 | + * <p>The shared objects body requires all transformations accessing the shared objects, i.e., |
| 35 | + * {@link SharedObjectsBodyResult#coLocatedTransformations}, to have same parallelism and can be |
| 36 | + * co-located. |
35 | 37 | */ |
36 | 38 | @Experimental |
37 | 39 | @FunctionalInterface |
38 | | -public interface SharedStorageBody extends Serializable { |
| 40 | +public interface SharedObjectsBody extends Serializable { |
39 | 41 |
|
40 | 42 | /** |
41 | | - * This method creates the subgraph for the shared storage body. |
| 43 | + * This method creates the subgraph for the shared objects body. |
42 | 44 | * |
43 | 45 | * @param inputs Input data streams. |
44 | 46 | * @return Result of the subgraph, including output data streams, data streams with access to |
45 | | - * the shared storage, and a mapping from share items to their owners. |
| 47 | + * the shared objects, and a mapping from share items to their owners. |
46 | 48 | */ |
47 | | - SharedStorageBodyResult process(List<DataStream<?>> inputs); |
| 49 | + SharedObjectsBodyResult process(List<DataStream<?>> inputs); |
48 | 50 |
|
49 | 51 | /** |
50 | | - * The result of a {@link SharedStorageBody}, including output data streams, data streams with |
51 | | - * access to the shared storage, and a mapping from descriptors of share items to their owners. |
| 52 | + * The result of a {@link SharedObjectsBody}, including output data streams, data streams with |
| 53 | + * access to the shared objects, and a mapping from descriptors of share items to their owners. |
52 | 54 | */ |
53 | 55 | @Experimental |
54 | | - class SharedStorageBodyResult { |
| 56 | + class SharedObjectsBodyResult { |
55 | 57 | /** A list of output streams. */ |
56 | 58 | private final List<DataStream<?>> outputs; |
57 | 59 |
|
58 | | - /** |
59 | | - * A list of data streams which access to the shared storage. All data streams in the list |
60 | | - * should implement {@link SharedStorageStreamOperator}. |
61 | | - */ |
62 | | - private final List<DataStream<?>> accessors; |
| 60 | + /** A list of {@link Transformation}s that should be co-located. */ |
| 61 | + private final List<Transformation<?>> coLocatedTransformations; |
63 | 62 |
|
64 | 63 | /** |
65 | 64 | * A mapping from descriptors of shared items to their owners. The owner is specified by |
66 | | - * {@link SharedStorageStreamOperator#getSharedStorageAccessorID()}, which must be kept |
67 | | - * unchanged for an instance of {@link SharedStorageStreamOperator}. |
| 65 | + * {@link SharedObjectsStreamOperator#getSharedObjectsAccessorID()}, which must be kept |
| 66 | + * unchanged for an instance of {@link SharedObjectsStreamOperator}. |
68 | 67 | */ |
69 | | - private final Map<ItemDescriptor<?>, SharedStorageStreamOperator> ownerMap; |
| 68 | + private final Map<ItemDescriptor<?>, SharedObjectsStreamOperator> ownerMap; |
70 | 69 |
|
71 | | - public SharedStorageBodyResult( |
| 70 | + public SharedObjectsBodyResult( |
72 | 71 | List<DataStream<?>> outputs, |
73 | | - List<DataStream<?>> accessors, |
74 | | - Map<ItemDescriptor<?>, SharedStorageStreamOperator> ownerMap) { |
| 72 | + List<Transformation<?>> coLocatedTransformations, |
| 73 | + Map<ItemDescriptor<?>, SharedObjectsStreamOperator> ownerMap) { |
75 | 74 | this.outputs = outputs; |
76 | | - this.accessors = accessors; |
| 75 | + this.coLocatedTransformations = coLocatedTransformations; |
77 | 76 | this.ownerMap = ownerMap; |
78 | 77 | } |
79 | 78 |
|
80 | 79 | public List<DataStream<?>> getOutputs() { |
81 | 80 | return outputs; |
82 | 81 | } |
83 | 82 |
|
84 | | - public List<DataStream<?>> getAccessors() { |
85 | | - return accessors; |
| 83 | + public List<Transformation<?>> getCoLocatedTransformations() { |
| 84 | + return coLocatedTransformations; |
86 | 85 | } |
87 | 86 |
|
88 | | - public Map<ItemDescriptor<?>, SharedStorageStreamOperator> getOwnerMap() { |
| 87 | + public Map<ItemDescriptor<?>, SharedObjectsStreamOperator> getOwnerMap() { |
89 | 88 | return ownerMap; |
90 | 89 | } |
91 | 90 | } |
|
0 commit comments