Skip to content

Conversation

@peihe
Copy link
Contributor

@peihe peihe commented May 16, 2016

This PR will make Dataflow streaming runner work with BoundedSources, such as TextIO and AvroIO.

@peihe peihe force-pushed the unbounded-read branch from d9f25a4 to 2e1c9d6 Compare May 16, 2016 20:10
@peihe
Copy link
Contributor Author

peihe commented May 16, 2016

R: @dpmills
R: @dhalperi
R: @tgroh

}

/**
* Returns a new {@link UnboundedReadFromBoundedSourceTest}.
Copy link
Member

Choose a reason for hiding this comment

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

Wrong ref

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

@peihe peihe force-pushed the unbounded-read branch from 6e84d46 to a0da764 Compare May 16, 2016 22:24
}

@Override
public boolean requiresDeduping() {
Copy link
Member

Choose a reason for hiding this comment

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

This is incomplete - You must provide an implementation of getCurrentRecordId() for requiresDeduping() to function (otherwise it will throw an exception in calls to getCurrentRecordId()). Probably right to use (element number, shardId) in this invocation. AFAIK, deduping is best-effort (not over the life of a pipeline), so this may produce duplicate elements in some cases.

This also does not solve the problem of progress - the reader may be discarded before it returns false to start() or advance() (as is done in the InProcessPipelineRunner), which may cause to the reader producing the same subset of elements and never completing the input. I believe reading the entire contents into an Iterable within start() and outputting (and then flattening) that Iterable will provide completion and progress guarantees, excepting the case in which not all elements fit in memory. Potentially the produced Iterable can be implemented as a channel back to the underlying BoundedReader (and thus lazily produce elements as TimestampedValues, which can be written to some channel and cleared out of memory if supported by the runner).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

removed requiresDeduping and getCurrentRecordId, since checkpoint is added.

@peihe peihe force-pushed the unbounded-read branch 2 times, most recently from c35fdf3 to 14cc999 Compare May 18, 2016 02:04
@peihe peihe changed the title Add toUnbounded() to get a Unbounded Read from a Bounded Read. Add UnboundedReadFromBoundedSource, and use it in Dataflow runner May 18, 2016
@peihe
Copy link
Contributor Author

peihe commented May 18, 2016

PTAL

(Jenkins is seems broken for unrelated reasons.)

@peihe peihe force-pushed the unbounded-read branch 2 times, most recently from cb15794 to d9ec83a Compare May 18, 2016 18:51
class Reader extends UnboundedReader<T> {
private final PipelineOptions options;

private @Nullable final List<TimestampedValue<T>> residualElements;
Copy link
Member

Choose a reason for hiding this comment

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

Avoid null - this is just empty if there are no residual elements.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

@tgroh
Copy link
Member

tgroh commented May 18, 2016

LGTM after null fix

@dpmills @dhalperi

public void finalizeCheckpoint() {}
}

private static class CheckpointCoder<T> extends AtomicCoder<Checkpoint<T>> {
Copy link
Contributor

Choose a reason for hiding this comment

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

This should be a StandardCoder; it is parameterized by elemCoder, so it is not atomic

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

@peihe
Copy link
Contributor Author

peihe commented May 18, 2016

PTAL

import javax.annotation.Nullable;

/**
* {@link PTransform} that performs a unbounded read from an {@link BoundedSource}.
Copy link
Contributor

Choose a reason for hiding this comment

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

This isn't a PTransform any more

Copy link
Contributor

Choose a reason for hiding this comment

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

Nevermind on this one, not sure what I was thinking.

@peihe peihe force-pushed the unbounded-read branch 3 times, most recently from 1017ce9 to 8588f6d Compare May 20, 2016 20:14
@peihe
Copy link
Contributor Author

peihe commented Jun 6, 2016

@dhalperi feedback?

public class UnboundedReadFromBoundedSource<T> extends PTransform<PInput, PCollection<T>> {
private final BoundedSource<T> source;

public UnboundedReadFromBoundedSource(BoundedSource<T> source) {
Copy link
Contributor

Choose a reason for hiding this comment

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

javadoc

Copy link
Contributor Author

Choose a reason for hiding this comment

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

added

@peihe
Copy link
Contributor Author

peihe commented Jun 21, 2016

Ready to review

}
}

<<<<<<< HEAD
Copy link
Contributor

Choose a reason for hiding this comment

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

?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

this was from an outdated diff

@dhalperi
Copy link
Contributor

LGTM

@dhalperi
Copy link
Contributor

Minor fixes, ping me when it's green to merge this and the other PR.

@peihe
Copy link
Contributor Author

peihe commented Jun 23, 2016

Addressed comments, and rebased for conflicts.

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;

import org.apache.beam.runners.core.UnboundedReadFromBoundedSource;
Copy link
Contributor

Choose a reason for hiding this comment

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

@kennknowles @davorbonaci adding this here would make the Dataflow runner depend on runners-core. This violates our prior assumption that only the service half of the runner should depend on runner core.

Opinions?

@dhalperi
Copy link
Contributor

Doesn't build, for checkstyle but also more fundamentally for the issue identified above. Let's discuss tomorrow.

<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>beam-runners-core-java</artifactId>
<scope>runtime</scope>
Copy link
Member

Choose a reason for hiding this comment

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

Discussed offline. I believe we have good options for short term and long term to avoid bringing back this dependency:

Longer term: Put the functionality in runners-core but only invoke it in the Dataflow service. If this turns out to be easy then we should do it right away.

Short-term: Put the functionality elsewhere, in one of:

  • Dataflow runner module. This is my preference. Right now it is really a matter of how the Dataflow runner works that it is necessary to have this adapter. If another runner decides that it wants to go this same route, then we might be in the longer term scenario anyhow.
  • SDK. Mostly harmless & we should move it out prior to a stable release.
  • Some io-core module that is not quite the grab bag that runners-core is slated to be.

This highlights the issue that there are really two needs for a utility library:

  1. Help implement the Beam model on the backend. This will generally impact a service which can be updated transparently and in an agile manner. It presupposes the service is aware of Beam constructs.
  2. Help to put together a translation from a Beam pipeline to an underlying backend. This will generally occur in SDK-adjacent client-side code, which cannot be updated easily.

We can conflate the two without much of a downside, as long as we shade on both sides, but it helps me to think of them separately. The main risk being that we come to have too thick a client that is hard to update. Or we could split them pretty easily. I propose we wait and see.

@dhalperi
Copy link
Contributor

LGTM. Will merge when green.

@asfgit asfgit closed this in 7745b92 Jun 24, 2016
dhalperi added a commit to dhalperi/beam that referenced this pull request Aug 23, 2016
@peihe peihe deleted the unbounded-read branch August 21, 2017 02:25
iemejia pushed a commit to iemejia/beam that referenced this pull request Jan 12, 2018
[BEAM-3121] Remove broken docker script and documentation
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.

5 participants