Skip to content

[BEAM-2926] Migrate to using a trivial multimap materialization within the Java SDK.#4011

Closed
lukecwik wants to merge 5 commits intoapache:masterfrom
lukecwik:side_input
Closed

[BEAM-2926] Migrate to using a trivial multimap materialization within the Java SDK.#4011
lukecwik wants to merge 5 commits intoapache:masterfrom
lukecwik:side_input

Conversation

@lukecwik
Copy link
Member

@lukecwik lukecwik commented Oct 18, 2017

Follow this checklist to help us incorporate your contribution quickly and easily:

  • Make sure there is a JIRA issue filed for the change (usually before you start working on it). Trivial changes like typos do not require a JIRA issue. Your pull request should address just this issue, without pulling in other changes.
  • Each commit in the pull request should have a meaningful subject line and body.
  • Format the pull request title like [BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replace BEAM-XXX with the appropriate JIRA issue.
  • Write a pull request description that is detailed enough to understand what the pull request does, how, and why.
  • Run mvn clean verify to make sure basic checks pass. A more thorough check will be performed on your pull request automatically.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

I want to get this reviewed before I start mucking with the Dataflow worker so I can update that internally after this approach has been agreed upon.

Review notes:

  • I dropped the "helpers" in PCollectionViewTesting that weren't very useful.
  • I also migrated off of tests calling PCollectionViews directly and instead constructing them through pc.apply(View.asYYY()) to simplify maintenance since PCollectionViews is internal only view View.asYYY is public.
  • I tried to migrate uses of PCollectionView.getCoderInternal()/PCollection.getWindowingStrategyInternal()/... to PCollectionView.getPCollection().getYYY() but was unable to because the PCollection stored within PCollectionView is transient and the majority of runners pull information from serialized PCollectionViews, not from proto round tripped views.
  • I loosened the types on PCollectionView so that they weren't so "Iterable" materialization focused.
  • I thought of creating a MultimapPCollectionView or something like that which narrowed the types of PCollectionView so it was easier to use PCollectionView instead of all the ? types and forced casts but that would have grown this PR to be so much larger.

@lukecwik lukecwik force-pushed the side_input branch 2 times, most recently from f9dff19 to f7240d8 Compare October 19, 2017 16:32
@lukecwik
Copy link
Member Author

R: @tgroh
CC: @kennknowles

I had a brief discussion with Ken about doing this high level approach since he had been the largest contributor to the PCollectionView interfaces from my memory. Ken, you might want to take a look at the high level interface changes.

public static List<Object> materializeValuesFor(
PTransform<?, ? extends PCollectionView<?>> viewTransformClass, Object ... values) {
List<Object> rval = new ArrayList<>();
// Currently all view materializations are the same where the data is shared underneath
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can this be any more specific than a PTransform, Object[] -> List<Object>? Narrowing the materialization (e.g. must be a Multimap, by name or by checkArgument) is fine.

A comment on what the method exists for would also be appreciated - my understanding of how this interacts with the existing ViewFns is that they're stored as the encoded objects in a singleton map from the void key; and in the future we'll do something smarter; this will be responsible for materializing those as well?

Copy link
Member Author

@lukecwik lukecwik Oct 19, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is for the future in case we have other materializations then just multimap so we can't give anything more specific then Object.

Your assumption is correct. This is to decouple the SDKs representation of how it chooses to materialize the different views with how the runner is testing them.

Once side inputs are being only fetched over the Fn API then this will not be the way to test things as really the Runner should be dealing just with storage and look up and not need to deal with the ViewFn at all.

// Generate the temporary-file prefix.
private PCollectionView<String> createTempFilePrefixView(PCollectionView<String> jobIdView) {
return ((PCollection<String>) jobIdView.getPCollection())
private PCollectionView<String> createTempFilePrefixView(
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can this change be separated out? It already LGTM, assuming that we don't ever expect to have separate prefixes per window.

Copy link
Member Author

@lukecwik lukecwik Oct 19, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Like part of another PR or just called out as a commit in this PR?

Currently just split to separate commit.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, that's fine.

// * null keys
// * null values
// * duplicate values
Multimap<Object, Object> multimap = ArrayListMultimap.create();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This can be a ListMultimap to force a bit more of the preconditions. Up to you.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The List part isn't important. Its just the ability to store null keys and values and duplicate values. For example HashMultimap doesn't support duplicate values and ImmutableMultimap doesn't support null keys/values.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wanted to use ListMultimap instead of generic Multimap to signal the non-set nature of the Multimap, as there's unfortunately no BagMultimap, but It's a minor nit given the commenting.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think the firstNotNull is necessary; Guava Multimaps return empty collections when there are no values associated with a key.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, updated.

return sideInput.getViewFn().apply(elements);
ViewFn<MultimapView, T> viewFn = (ViewFn<MultimapView, T>) view.getViewFn();
Coder<?> keyCoder = ((KvCoder<?, ?>) view.getCoderInternal()).getKeyCoder();
return viewFn.apply(InMemoryMultimapSideInputView.fromIterable(keyCoder, (Iterable) elements));
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can/do we have an outstanding task to update the handler to use a more appropriate storage type, and/or is there a more appropriate state type?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The issue from swapping away from value state is that not all runners may support more complex state types and value state effectively forces using an in memory representation.

@SuppressWarnings("unchecked") Iterable<KV<?, ?>> elements = Iterables.transform(
(Iterable<WindowedValue<KV<?, ?>>>) viewContents.getUnchecked(
PCollectionViewWindow.of(view, window)).get(),
new Function<WindowedValue<KV<?, ?>>, KV<?, ?>>() {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of storing the input Multimap's entries as an iterable and converting back, can we just store the multimap directly? If that's too large for this change, file a followup jira.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The plan was to eventually improve these representations (or have runner authors make these changes), see https://issues.apache.org/jira/browse/BEAM-3080 for the DirectRunner/ULR.

This was just to get the change in since all the runners here just do a trivial multimap -> iterable -> multimap for all the types anyways.

public class SideInputInitializer<ElemT, ViewT, W extends BoundedWindow>
implements BroadcastVariableInitializer<WindowedValue<ElemT>, Map<BoundedWindow, ViewT>> {
public class SideInputInitializer<ViewT>
implements BroadcastVariableInitializer<WindowedValue<?>, Map<BoundedWindow, ViewT>> {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can this be WindowedValues of KVs, or is that not possible with the module's type signatures?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm specifically not narrowing the type because KV<> is for the multimap materialization. When/if this SideInputInitializer supports a different URN, that KV is unlikely to be true anymore.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same with a @Before method

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No action required, but these number changes are a real downer.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(nit) can we just use contains?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@lukecwik
Copy link
Member Author

Run Java PreCommit

Copy link
Member

@tgroh tgroh left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

// * null keys
// * null values
// * duplicate values
Multimap<Object, Object> multimap = ArrayListMultimap.create();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wanted to use ListMultimap instead of generic Multimap to signal the non-set nature of the Multimap, as there's unfortunately no BagMultimap, but It's a minor nit given the commenting.

// Generate the temporary-file prefix.
private PCollectionView<String> createTempFilePrefixView(PCollectionView<String> jobIdView) {
return ((PCollection<String>) jobIdView.getPCollection())
private PCollectionView<String> createTempFilePrefixView(
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, that's fine.

@lukecwik lukecwik force-pushed the side_input branch 3 times, most recently from ba39ffd to 76c4fc0 Compare October 25, 2017 21:02
@lukecwik
Copy link
Member Author

Run Dataflow ValidatesRunner

@lukecwik
Copy link
Member Author

Run Apex ValidatesRunner

@lukecwik
Copy link
Member Author

Run Spark ValidatesRunner

@lukecwik
Copy link
Member Author

Run Flink ValidatesRunner

@lukecwik
Copy link
Member Author

Run Gearpump ValidatesRunner

@asfgit
Copy link

asfgit commented Nov 13, 2017

SUCCESS

--none--

4 similar comments
@asfgit
Copy link

asfgit commented Nov 13, 2017

SUCCESS

--none--

@asfgit
Copy link

asfgit commented Nov 13, 2017

SUCCESS

--none--

@asfgit
Copy link

asfgit commented Nov 13, 2017

SUCCESS

--none--

@asfgit
Copy link

asfgit commented Nov 13, 2017

SUCCESS

--none--

@asfgit
Copy link

asfgit commented Nov 13, 2017

FAILURE

--none--

2 similar comments
@asfgit
Copy link

asfgit commented Nov 13, 2017

FAILURE

--none--

@asfgit
Copy link

asfgit commented Nov 13, 2017

FAILURE

--none--

@lukecwik
Copy link
Member Author

Run Dataflow ValidatesRunner

@lukecwik
Copy link
Member Author

Run Spark ValidatesRunner

@lukecwik
Copy link
Member Author

Run Apex ValidatesRunner

@lukecwik
Copy link
Member Author

Run Gearpump ValidatesRunner

@lukecwik
Copy link
Member Author

Run Flink ValidatesaRunner

@asfgit
Copy link

asfgit commented Nov 14, 2017

SUCCESS

--none--

@lukecwik
Copy link
Member Author

Run Flink ValidatesRunner

@asfgit
Copy link

asfgit commented Nov 14, 2017

SUCCESS

--none--

@asfgit
Copy link

asfgit commented Nov 14, 2017

FAILURE

--none--

@asfgit
Copy link

asfgit commented Nov 14, 2017

SUCCESS

--none--

2 similar comments
@asfgit
Copy link

asfgit commented Nov 14, 2017

SUCCESS

--none--

@asfgit
Copy link

asfgit commented Nov 14, 2017

SUCCESS

--none--

@asfgit
Copy link

asfgit commented Nov 14, 2017

FAILURE

--none--

1 similar comment
@asfgit
Copy link

asfgit commented Nov 14, 2017

FAILURE

--none--

@asfgit
Copy link

asfgit commented Nov 15, 2017

SUCCESS

--none--

@lukecwik
Copy link
Member Author

Run Dataflow ValidatesRunner

@lukecwik
Copy link
Member Author

Run Apex ValidatesRunner

@lukecwik
Copy link
Member Author

Run Spark ValidatesRunner

@lukecwik
Copy link
Member Author

Run Flink ValidatesRunner

@lukecwik
Copy link
Member Author

Run Gearpump ValidatesRunner

@asfgit
Copy link

asfgit commented Nov 15, 2017

SUCCESS

--none--

6 similar comments
@asfgit
Copy link

asfgit commented Nov 15, 2017

SUCCESS

--none--

@asfgit
Copy link

asfgit commented Nov 15, 2017

SUCCESS

--none--

@asfgit
Copy link

asfgit commented Nov 15, 2017

SUCCESS

--none--

@asfgit
Copy link

asfgit commented Nov 15, 2017

SUCCESS

--none--

@asfgit
Copy link

asfgit commented Nov 15, 2017

SUCCESS

--none--

@asfgit
Copy link

asfgit commented Nov 15, 2017

SUCCESS

--none--

@lukecwik
Copy link
Member Author

@tgroh I'm going to merge this as is to avoid the continued Dataflow worker dance.

Feel free to add additional comments for 341521a and I'll apply them in a follow up PR.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants

Comments