AW: Guarantees for object reuse modes and documentation

2016-03-03 Thread Fabian Hueske
Hi Ken, yes, that’s exactly how it work in Flink as well. The object reuse move dues not affect how chaining is done. Best, Fabian Von: Ken Krugler Gesendet: Freitag, 4. März 2016 01:54 An: dev@flink.apache.org Betreff: RE: Guarantees for object reuse modes and documentation Hi Gábor, When

RE: Guarantees for object reuse modes and documentation

2016-03-03 Thread Ken Krugler
ng is done, which prevents a problem with output tuple re-use. Is this the same model used by Flink? Thanks for clarifying, -- Ken > From: Gábor Gévay > Sent: February 20, 2016 4:04:09am PST > To: dev@flink.apache.org > Subject: Re: Guarantees for object reuse modes and documentation &g

Re: Guarantees for object reuse modes and documentation

2016-02-25 Thread Fabian Hueske
changes to this in the upstream operation >>> obviously bork the cached data. >>> > >>> > Net-net is that it we wanted a way to find out, from inside an >>> operation, whether we needed to make a deep copy of the output Tuple. But >>> that doesn&#

Re: Guarantees for object reuse modes and documentation

2016-02-24 Thread Fabian Hueske
; operation, whether we needed to make a deep copy of the output Tuple. But >> that doesn't exist (yet), so we have some utility code to check if a deep >> copy is needed (non-primitive types), and if so then it auto-clones the >> Tuple. Which isn't very efficient, but

Re: Guarantees for object reuse modes and documentation

2016-02-24 Thread Fabian Hueske
he > Tuple. Which isn't very efficient, but for most of our workflows we only > have primitive types. > > > > -- Ken > > > >> From: Fabian Hueske > >> Sent: February 17, 2016 9:17:27am PST > >> To: dev@flink.apache.org > >> Subject: Gua

Re: Guarantees for object reuse modes and documentation

2016-02-20 Thread Gábor Gévay
> very efficient, but for most of our workflows we only have primitive types. > > -- Ken > >> From: Fabian Hueske >> Sent: February 17, 2016 9:17:27am PST >> To: dev@flink.apache.org >> Subject: Guarantees for object reuse modes and documentation >> >

RE: Guarantees for object reuse modes and documentation

2016-02-19 Thread Ken Krugler
ske > Sent: February 17, 2016 9:17:27am PST > To: dev@flink.apache.org > Subject: Guarantees for object reuse modes and documentation > > Hi, > > > > Flink's DataSet API features a configuration parameter called > enableObjectReuse(). If activated, Flink's runtime

Re: Guarantees for object reuse modes and documentation

2016-02-18 Thread Greg Hogan
Hi Fabian, I would only add to your citations Stephan's comment [1] concerning the design, implementation, and use of object reuse. I see two separate concerns addressed in code. First, as Stephan noted, for certain classes deserialization is sufficiently expensive relative to object creation and

Re: Guarantees for object reuse modes and documentation

2016-02-18 Thread Till Rohrmann
Judging from our chaining condition ds.getPushChainDriverClass() != null && !(pred instanceof NAryUnionPlanNode) &&// first op after union is stand-alone, because union is merged !(pred instanceof BulkPartialSolutionPlanNode) &&// partial solution merges anyways !(pred instanceof WorksetPl

Re: Guarantees for object reuse modes and documentation

2016-02-18 Thread Fabian Hueske
Thanks Matthias. Maybe I should clarify, that I do not want to change the guarantees for the enableObjectReuse mode, but for the disableObjectReuse mode. The rules for the enableObjectReuse mode should remain the same. 2016-02-18 9:37 GMT+01:00 Matthias J. Sax : > Hi, > > I like Fabian's proposa

Re: Guarantees for object reuse modes and documentation

2016-02-18 Thread Matthias J. Sax
Hi, I like Fabian's proposal. The idea of object reuse is performance gain, and we should not sacrifice this. Even more important is that the rules are easy to understand! -Matthias On 02/17/2016 06:17 PM, Fabian Hueske wrote: > Hi, > > > > Flink's DataSet API features a configuration parame

Guarantees for object reuse modes and documentation

2016-02-17 Thread Fabian Hueske
Hi, Flink's DataSet API features a configuration parameter called enableObjectReuse(). If activated, Flink's runtime will create fewer objects which results in better performance and lower garbage collection overhead. Depending on whether the configuration switch is enabled or not, user function