Conversation
|
R: @kennknowles |
|
Refer to this link for build results (access rights to CI server needed): Failed Tests: 1beam_PreCommit_Java_MavenInstall/org.apache.beam:beam-sdks-java-core: 1--none-- |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
b371783 to
0783c3e
Compare
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
This takes SDK objects and assigns IDs to them. It is effectively a ComponentsBuilder context where a component is referred to by the Java object which is being translated, rather than by an opaque string or protocol buffer. The Builder for SdkComponents exposes mutation methods to add components. The actual SdkComponents implementation is deeply immutable, to signal that during actual conversion to/from protos should assume that all elements are already recorded as a component.
|
Refer to this link for build results (access rights to CI server needed): |
|
R: @lukecwik |
|
For reference, I have a branch https://github.com/tgroh/beam/tree/coders_utilities that is approximately where I expect this to go, after implementing each type of component translation. |
| * <p>Uses the full name of the transform as the id. | ||
| */ | ||
| public String getTransformId(AppliedPTransform<?, ?, ?> pTransform) { | ||
| String existing = transformIds.get(pTransform); |
There was a problem hiding this comment.
I spoke with Ken and he gave me the impression that that the id in the components maps was some kind of UUID and not equivalent or related to the unique name.
@kennknowles Comment?
There was a problem hiding this comment.
It is independent, but a fine choice. The "id" is just a pointer into the map and has no other significance. The unique name is required for e.g. identifying state associated with the transform across pipeline updates. (and whatever other programmatic uses a runner might have for a unique name)
There was a problem hiding this comment.
This is easy to change to be unrelated. Done.
The reason I did this was a failure to realize the independence of unique_name and the keys in the components.
There was a problem hiding this comment.
I would stick with the longer readable name that you had, its easy for a runner to walk the pipeline and produce a version of the pipeline with short names if it thinks that it is valuable.
| * <p>Uses the full name of the transform as the id. | ||
| */ | ||
| public String getTransformId(AppliedPTransform<?, ?, ?> pTransform) { | ||
| String existing = transformIds.get(pTransform); |
There was a problem hiding this comment.
This assumes that the user never overrides equals when creating a subclass of pTransform.
Is instance equality all you need or more? (Consider using Equivalence.Wrapper)
There was a problem hiding this comment.
Applied Transforms are only equal if they have identical names, among other things. Given that names must be unique in the graph, this is sufficient to ensure repeated applications show up multiple times.
| * <p>Uses a unique name based on the result of {@link PCollection#getName()}. | ||
| */ | ||
| public String getPCollectionId(PCollection<?> pCollection) { | ||
| String existing = pCollectionIds.get(pCollection); |
There was a problem hiding this comment.
Make PCollection final to prevent overriding and messing up the equals method or use an equivalence wrapper?
There was a problem hiding this comment.
It's impossible to construct a PCollection subclass, as it has no visible constructors.
| * SdkComponents}. | ||
| */ | ||
| public String getWindowingStrategyId(WindowingStrategy<?, ?> windowingStrategy) { | ||
| String existing = windowingStrategyIds.get(windowingStrategy); |
There was a problem hiding this comment.
Make windowing strategy final or use an equivalence wrapper?
| * <p>Uses the full name of the transform as the id. | ||
| */ | ||
| public String getTransformId(AppliedPTransform<?, ?, ?> pTransform) { | ||
| String existing = transformIds.get(pTransform); |
There was a problem hiding this comment.
I would stick with the longer readable name that you had, its easy for a runner to walk the pipeline and produce a version of the pipeline with short names if it thinks that it is valuable.
| * returning the id of the {@link Coder} in this {@link SdkComponents}. | ||
| */ | ||
| public String getCoderId(Coder<?> coder) { | ||
| String existing = coderIds.get(coder); |
| * | ||
| * <p>Uses the full name of the transform as the id. | ||
| */ | ||
| public String getTransformId(AppliedPTransform<?, ?, ?> pTransform) { |
There was a problem hiding this comment.
registerPTransform
Update comment to something along:
registers a ptransform assigning a unique id, multiple registrations of the same ptransform return the id of the original transform.
Ditto for the rest like:
registerCoder
registerWindowingStrategy
There was a problem hiding this comment.
+1 specifically because you are building an object that you will convert to proto, hence the set of all registered things is observable.
Phrasing on the spec: multiple registration of the same transform yield the same id
| * PCollection PCollections}, and {@link PTransform PTransforms}. | ||
| */ | ||
| @Experimental | ||
| public RunnerApi.Components toComponents() { |
There was a problem hiding this comment.
Would it better be to just implement a PipelineVisitor that converts from a "java" pipeline to the proto representation, this way you don't need to expose public methods and are effectively walking the graph?
There was a problem hiding this comment.
~ish. We should not use these from outside of this package (SomeComponent.toProto should use this as the underlying way to construct a MessageWithComponents), but we want to be able to convert arbitrary components without the entire pipeline.
|
Refer to this link for build results (access rights to CI server needed): Failed Tests: 1beam_PreCommit_Java_MavenInstall/org.apache.beam:beam-sdks-java-core: 1--none-- |
|
Refer to this link for build results (access rights to CI server needed): |
tgroh
left a comment
There was a problem hiding this comment.
I've also made this class mostly package private, and moved it to runners-core-construction so the Protos aren't exposed to users.
| * <p>Uses a unique name based on the result of {@link PCollection#getName()}. | ||
| */ | ||
| public String getPCollectionId(PCollection<?> pCollection) { | ||
| String existing = pCollectionIds.get(pCollection); |
There was a problem hiding this comment.
It's impossible to construct a PCollection subclass, as it has no visible constructors.
| * SdkComponents}. | ||
| */ | ||
| public String getWindowingStrategyId(WindowingStrategy<?, ?> windowingStrategy) { | ||
| String existing = windowingStrategyIds.get(windowingStrategy); |
| * PCollection PCollections}, and {@link PTransform PTransforms}. | ||
| */ | ||
| @Experimental | ||
| public RunnerApi.Components toComponents() { |
There was a problem hiding this comment.
~ish. We should not use these from outside of this package (SomeComponent.toProto should use this as the underlying way to construct a MessageWithComponents), but we want to be able to convert arbitrary components without the entire pipeline.
| * <p>Uses the full name of the transform as the id. | ||
| */ | ||
| public String getTransformId(AppliedPTransform<?, ?, ?> pTransform) { | ||
| String existing = transformIds.get(pTransform); |
There was a problem hiding this comment.
Applied Transforms are only equal if they have identical names, among other things. Given that names must be unique in the graph, this is sufficient to ensure repeated applications show up multiple times.
| * <p>Uses the full name of the transform as the id. | ||
| */ | ||
| public String getTransformId(AppliedPTransform<?, ?, ?> pTransform) { | ||
| String existing = transformIds.get(pTransform); |
| * | ||
| * <p>Uses the full name of the transform as the id. | ||
| */ | ||
| public String getTransformId(AppliedPTransform<?, ?, ?> pTransform) { |
| * returning the id of the {@link Coder} in this {@link SdkComponents}. | ||
| */ | ||
| public String getCoderId(Coder<?> coder) { | ||
| String existing = coderIds.get(coder); |
|
Refer to this link for build results (access rights to CI server needed): Build result: FAILURE[...truncated 808.64 KB...] at hudson.remoting.UserRequest.perform(UserRequest.java:153) at hudson.remoting.UserRequest.perform(UserRequest.java:50) at hudson.remoting.Request$2.run(Request.java:336) at hudson.remoting.InterceptingExecutorService$1.call(InterceptingExecutorService.java:68) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745)Caused by: org.apache.maven.plugin.MojoExecutionException: Dependency problems found at org.apache.maven.plugins.dependency.analyze.AbstractAnalyzeMojo.execute(AbstractAnalyzeMojo.java:261) at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo(DefaultBuildPluginManager.java:134) at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:208) ... 31 more2017-04-10T20:29:02.128 [ERROR] 2017-04-10T20:29:02.128 [ERROR] Re-run Maven using the -X switch to enable full debug logging.2017-04-10T20:29:02.128 [ERROR] 2017-04-10T20:29:02.128 [ERROR] For more information about the errors and possible solutions, please read the following articles:2017-04-10T20:29:02.129 [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoExecutionException2017-04-10T20:29:02.129 [ERROR] 2017-04-10T20:29:02.129 [ERROR] After correcting the problems, you can resume the build with the command2017-04-10T20:29:02.130 [ERROR] mvn -rf :beam-runners-core-construction-javachannel stoppedSetting status of e46df61 to FAILURE with url https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/9365/ and message: 'Build finished. 'Using context: Jenkins: Maven clean install--none-- |
lukecwik
left a comment
There was a problem hiding this comment.
Minor comments and then LGTM
| * present, returning the id of the provided {@link PCollection} in this {@link SdkComponents}. | ||
| * Registers the provided {@link PCollection} into this {@link SdkComponents}, returning a unique | ||
| * ID for the {@link PCollection}. Multiple calls to {@link #registerPCollection(PCollection)} | ||
| * with the same {@link PCollection}will return the same unique ID. |
| * ID for the {@link PCollection}. Multiple calls to {@link #registerPCollection(PCollection)} | ||
| * with the same {@link PCollection}will return the same unique ID. | ||
| * | ||
| * <p>Uses a unique name based on the result of {@link PCollection#getName()}. |
There was a problem hiding this comment.
I would drop the bit about uses a unique name, since these are internal details. For all intensive purposes your returning random ids.
| * SdkComponents}. | ||
| * Registers the provided {@link WindowingStrategy} into this {@link SdkComponents}, returning a | ||
| * unique ID for the {@link WindowingStrategy}. Multiple calls to {@link | ||
| * #registerWindowingStrategy(WindowingStrategy)} with the equal {@link WindowingStrategy |
There was a problem hiding this comment.
with the equal -> with equal
WindowingStrategies -> windowing strategies
| * Puts the provided {@link Coder} into this {@link SdkComponents} if it is not already present, | ||
| * returning the id of the {@link Coder} in this {@link SdkComponents}. | ||
| * Registers the provided {@link Coder} into this {@link SdkComponents}, returning a unique ID for | ||
| * the {@link Coder}. Multiple calls to {@link #registerCoder(Coder)} with the identical {@link |
There was a problem hiding this comment.
with the identical -> with identical
|
Method names updated everywhere, comments updated everywhere. |
|
Refer to this link for build results (access rights to CI server needed): |
Be sure to do all of the following to help us incorporate your contribution
quickly and easily:
[BEAM-<Jira issue #>] Description of pull requestmvn clean verify. (Even better, enableTravis-CI on your fork and ensure the whole test matrix passes).
<Jira issue #>in the title with the actual Jira issuenumber, if there is one.
Individual Contributor License Agreement.
This takes SDK objects and assigns IDs to them. It is effectively a
ComponentsBuilder context where a component is referred to by the
Java object which is being translated, rather than by an opaque
string or protocol buffer.