No, the internal behavior is not documented right now. But I think it will
change before the 1.0 release.
> On 30 Dec 2015, at 18:44, Brian Chhun wrote:
>
> Thanks for the clarification. Is there a resource besides the code that has
> these kinds of things documented? Understandable if there isn't much out
> there yet and that these things are still in flux.
>
> On Wed, Dec 30, 2015 at 11:14 AM, Aljoscha Krettek
> wrote:
> Yes, this is correct right now. It should not be too hard to add the
> pre-aggregation behavior for fold, however.
> > On 30 Dec 2015, at 17:31, Brian Chhun wrote:
>
> >
> > Hi All,
> >
> > Are certain considerations when using these functions on windowed streams?
> >
> > From reading the code, it looks using reduce (or another aggregation
> > function) on a windowed stream will pre-aggregate the result value as
> > elements are added to the window, keeping the size of window constant. On
> > the other hand, the fold function will accumulate elements into the window
> > and wait until the window is fired before computing the aggregation. Does
> > this sound correct?
> >
> > On Thu, Nov 19, 2015 at 1:27 PM, Stephan Ewen wrote:
> > Hi Ron!
> >
> > Yes, we had to change a few things in the API between 0.9 and 0.10. The API
> > in 0.9 had quite a few problems. This one now looks good, we are confident
> > that it will stay.
> >
> > Greetings,
> > Stephan
> >
> >
> > On Thu, Nov 19, 2015 at 8:15 PM, Ron Crocker wrote:
> > Thanks Stephan, that helps quite a bit. Looks like another one of those API
> > changes that I'll be struggling with for a little bit.
> >
> > On Thu, Nov 19, 2015 at 10:40 AM, Stephan Ewen wrote:
> > Hi Ron!
> >
> > You are right, there is a copy/paste error in the docs, it should be a
> > FoldFunction that is passed to fold(), not a ReduceFunction.
> >
> > In Flink-0.10, the FoldFunction is only available on
> >
> > - KeyedStream
> > (https://ci.apache.org/projects/flink/flink-docs-release-0.10/api/java/org/apache/flink/streaming/api/datastream/KeyedStream.html#fold(R,%20org.apache.flink.api.common.functions.FoldFunction))
> >
> > - WindowedStream
> > (https://ci.apache.org/projects/flink/flink-docs-release-0.10/api/java/org/apache/flink/streaming/api/datastream/WindowedStream.html#fold(R,%20org.apache.flink.api.common.functions.FoldFunction,%20org.apache.flink.api.common.typeinfo.TypeInformation))
> >
> > In most cases, you probably want the variant on the WindowedStream, if you
> > aggregate values over time.
> >
> >
> >
> > To the difference between fold() and reduce(): It is very subtle. The fold
> > function can also convert to another type whenever it integrates a new
> > element.
> >
> > Here is an example (with lists, not streams, but same principle).
> >
> >
> >
> > ReduceFunction {
> >
> > public Integer reduce(Integer a, Integer b) { return a + b; }
> > }
> >
> > [1, 2, 3, 4, 5] -> reduce() means: 1 + 2) + 3) + 4) + 5) = 15
> >
> >
> >
> > FoldFunction {
> >
> > public String fold(String current, Integer i) { return current +
> > String.valueOf(i); }
> > }
> >
> > [1, 2, 3, 4, 5] -> fold("start-") means: ("start-" + 1) + 2) + 3) + 4)
> > + 5) = "start-12345" (as a String)
> >
> >
> > I hope that example illustrates the difference.
> >
> >
> > Greetings,
> > Stephan
> >
> >
> > On Thu, Nov 19, 2015 at 7:00 PM, Ron Crocker wrote:
> > Hi Fabian -
> >
> > Thanks Fabian, that is a helpful description.
> >
> > That document WAS my source of information and it seems to also be the
> > source of my confusion. Further, it appears to be wrong - there is a
> > FoldFunction
> > (https://ci.apache.org/projects/flink/flink-docs-release-0.10/api/java/org/apache/flink/api/common/functions/FoldFunction.html)
> > that should be passed into fold()?
> >
> > Separate note: fold() doesn't appear in the javadocs for 0.10.0 DataStream
> > (see
> > https://ci.apache.org/projects/flink/flink-docs-release-0.10/api/java/org/apache/flink/streaming/api/datastream/DataStream.html).
> > So this made me look in the freshly-downloaded flink-streaming-java:0.10.0
> > and fold() does not appear in
> > org.apache.flink.streaming.api.datastream.DataStream either. Am I looking
> > in the wrong place for it? In 0.9.1, it's located in that same class with
> > this signature: fold(R initialValue, FoldFunction folder).
> >
> > Ron
> >
> > On Wed, Nov 18, 2015 at 9:39 AM, Fabian Hueske wrote:
> > Hi Ron,
> >
> > Have you checked:
> > https://ci.apache.org/projects/flink/flink-docs-release-0.10/apis/streaming_guide.html#transformations?
> >
> > Fold is like reduce, except that you define a start element (of a different
> > type than the input type) and the result type is the type of the initial
> > value. In reduce, the result type must be identical to the input type.
> >
>