flink-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Gábor Horváth <xazax....@gmail.com>
Subject Re: GSoC Project Proposal Draft: Code Generation in Serializers
Date Wed, 20 Apr 2016 14:18:56 GMT
On the second thought I think you are right. I had the impression that
there is cyclic dependency between TypeInformation and the serializers but
that is not the case. So there is no rewrite needed for TypeInformation in
order to be able to use Scala for serializers.

According to the proposal unless someone utilize the annotations the
generated serializers would be compatible to the current ones. There could
be a configuration option whether to try to make the layout more compact
based on annotations.

On 20 April 2016 at 16:03, Fabian Hueske <fhueske@gmail.com> wrote:

> Why would you need to rewrite the TypeInformation in Scala?
> I think we need a way to replace Serializer implementations anyway unless
> the generated serializers are compatible to the current ones.
>
> 2016-04-20 15:53 GMT+02:00 Gábor Horváth <xazax.hun@gmail.com>:
>
> > Hi Fabian,
> >
> > I agree that it would be awesome to move this to its own module/plugin.
> > However in order to be able to write the code generation in Scala I would
> > need to rewrite the type information to use Scala as well. I think I will
> > not
> > have time to do this during the summer, so I think I will stick to Java
> and
> > this modularization can be done later.
> >
> > Thanks,
> > Gábor
> >
> > On 19 April 2016 at 11:50, Fabian Hueske <fhueske@gmail.com> wrote:
> >
> > > Hi Gabor,
> > >
> > > you are right, a codegen serializer module would depend on flink-core
> and
> > > in the current design flink-core would need to know about the type
> infos
> > /
> > > serializers / comparators.
> > >
> > > Decoupling implementations of type info, serializers, and comparators
> > from
> > > flink-core and resolving the cyclic dependency would be what the plugin
> > > architecture would be for.
> > > Maybe this can be done by some mechanism to dynamically load
> > > TypeInformations for types with overridden serializers / comparators.
> > > This would require some design document and discussion in the
> community.
> > >
> > > Cheers, Fabian
> > >
> > >
> > >
> > >
> > >
> > > 2016-04-18 21:19 GMT+02:00 Gábor Horváth <xazax.hun@gmail.com>:
> > >
> > > > Unfortunately making code generation a separate module would
> introduce
> > > > cyclic dependency.
> > > > Code generation requires the TypeInfo which is available in
> flink-core
> > > and
> > > > flink-core requires
> > > > the generated serializers from the code generation module. Do you
> have
> > a
> > > > solution for this?
> > > >
> > > > I think if we can come up with a solution I will implement it as a
> > > separate
> > > > Scala module
> > > > otherwise I will stick to Java.
> > > >
> > > > BR,
> > > > Gábor
> > > >
> > > > On 18 April 2016 at 12:40, Fabian Hueske <fhueske@gmail.com> wrote:
> > > >
> > > > > +1 for not mixing Java and Scala in flink-core.
> > > > >
> > > > > Maybe it makes sense to implement the code generated serializers
/
> > > > > comparators as a separate module which can be plugged-in. This
> could
> > be
> > > > > pure Scala.
> > > > > In general, I think it would be good to have some kind of "version
> > > > > management" for serializers in place. With features such as
> > safepoints
> > > > that
> > > > > depend on the implementation of serializers, it would be good to
> > have a
> > > > > mechanism to switch between implementations.
> > > > >
> > > > > Best, Fabian
> > > > >
> > > > > 2016-04-18 10:01 GMT+02:00 Chiwan Park <chiwanpark@apache.org>:
> > > > >
> > > > > > Yes, I know Janino is a pure Java project. I meant if we add
> Scala
> > > code
> > > > > to
> > > > > > flink-core, we should add Scala dependency to flink-core and
it
> > could
> > > > be
> > > > > > confusing.
> > > > > >
> > > > > > Regards,
> > > > > > Chiwan Park
> > > > > >
> > > > > > > On Apr 18, 2016, at 2:49 PM, Márton Balassi <
> > > > balassi.marton@gmail.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > Chiwan, just to clarify Janino is a Java project. [1]
> > > > > > >
> > > > > > > [1] https://github.com/aunkrig/janino
> > > > > > >
> > > > > > > On Mon, Apr 18, 2016 at 3:40 AM, Chiwan Park <
> > > chiwanpark@apache.org>
> > > > > > wrote:
> > > > > > >
> > > > > > >> I prefer to avoid Scala dependencies in flink-core.
If
> > flink-core
> > > > > > includes
> > > > > > >> Scala dependencies, Scala version suffix (_2.10 or
_2.11)
> should
> > > be
> > > > > > added.
> > > > > > >> I think that users could be confused.
> > > > > > >>
> > > > > > >> Regards,
> > > > > > >> Chiwan Park
> > > > > > >>
> > > > > > >>> On Apr 17, 2016, at 3:49 PM, Márton Balassi <
> > > > > balassi.marton@gmail.com>
> > > > > > >> wrote:
> > > > > > >>>
> > > > > > >>> Hi Gábor,
> > > > > > >>>
> > > > > > >>> I think that adding the Janino dep to flink-core
should be
> > fine,
> > > as
> > > > > it
> > > > > > >> has
> > > > > > >>> quite slim dependencies [1,2] which are generally
orthogonal
> to
> > > > > Flink's
> > > > > > >>> main dependency line (also it is already used elsewhere).
> > > > > > >>>
> > > > > > >>> As for mixing Scala code that is used from the
Java parts of
> > the
> > > > same
> > > > > > >> maven
> > > > > > >>> module I am skeptical. We have seen IDE compilation
issues
> with
> > > > > > projects
> > > > > > >>> using this setup and have decided that the community-wide
> > > potential
> > > > > IDE
> > > > > > >>> setup pain outweighs the individual implementation
> convenience
> > > with
> > > > > > >> Scala.
> > > > > > >>>
> > > > > > >>> [1]
> > > > > > >>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://repo1.maven.org/maven2/org/codehaus/janino/janino-parent/2.7.8/janino-parent-2.7.8.pom
> > > > > > >>> [2]
> > > > > > >>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://repo1.maven.org/maven2/org/codehaus/janino/janino/2.7.8/janino-2.7.8.pom
> > > > > > >>>
> > > > > > >>> On Sat, Apr 16, 2016 at 5:51 PM, Gábor Horváth
<
> > > > xazax.hun@gmail.com>
> > > > > > >> wrote:
> > > > > > >>>
> > > > > > >>>> Hi!
> > > > > > >>>>
> > > > > > >>>> Table API already uses code generation and
the Janino
> compiler
> > > > [1].
> > > > > Is
> > > > > > >> it a
> > > > > > >>>> dependency that is ok to add to flink-core?
In case it is
> ok,
> > I
> > > > > think
> > > > > > I
> > > > > > >>>> will use the same in order to be consistent
with the other
> > code
> > > > > > >> generation
> > > > > > >>>> efforts.
> > > > > > >>>>
> > > > > > >>>> I started to look at the Table API code generation
[2] and
> it
> > > uses
> > > > > > Scala
> > > > > > >>>> extensively. There are several Scala features
that can make
> > Java
> > > > > code
> > > > > > >>>> generation easier such as pattern matching
and string
> > > > > interpolation. I
> > > > > > >> did
> > > > > > >>>> not see any Scala code in flink-core yet. Is
it ok to
> > implement
> > > > the
> > > > > > code
> > > > > > >>>> generation inside the flink-core using Scala?
> > > > > > >>>>
> > > > > > >>>> Regards,
> > > > > > >>>> Gábor
> > > > > > >>>>
> > > > > > >>>> [1] http://unkrig.de/w/Janino
> > > > > > >>>> [2]
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala
> > > > > > >>>>
> > > > > > >>>> On 18 March 2016 at 19:37, Gábor Horváth
<
> xazax.hun@gmail.com
> > >
> > > > > wrote:
> > > > > > >>>>
> > > > > > >>>>> Thank you! I finalized the project.
> > > > > > >>>>>
> > > > > > >>>>>
> > > > > > >>>>> On 18 March 2016 at 10:29, Márton Balassi
<
> > > > > balassi.marton@gmail.com>
> > > > > > >>>>> wrote:
> > > > > > >>>>>
> > > > > > >>>>>> Thanks Gábor, now I also see it on
the internal GSoC
> > > interface.
> > > > I
> > > > > > have
> > > > > > >>>>>> indicated that I wish to mentor your
project, I think you
> > can
> > > > hit
> > > > > > >>>> finalize
> > > > > > >>>>>> on your project there.
> > > > > > >>>>>>
> > > > > > >>>>>> On Mon, Mar 14, 2016 at 11:16 AM, Gábor
Horváth <
> > > > > > xazax.hun@gmail.com>
> > > > > > >>>>>> wrote:
> > > > > > >>>>>>
> > > > > > >>>>>>> Hi,
> > > > > > >>>>>>>
> > > > > > >>>>>>> I have updated this draft to include
preliminary
> > benchmarks,
> > > > > > >> mentioned
> > > > > > >>>>>> the
> > > > > > >>>>>>> interaction of annotations with
savepoints, extended it
> > with
> > > a
> > > > > > >>>> timeline,
> > > > > > >>>>>>> and some notes about scala case
classes.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Regards,
> > > > > > >>>>>>> Gábor
> > > > > > >>>>>>>
> > > > > > >>>>>>> On 9 March 2016 at 16:12, Gábor
Horváth <
> > xazax.hun@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >>>>>>>
> > > > > > >>>>>>>> Hi!
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> As far as I can see the formatting
was not correct in my
> > > > > previous
> > > > > > >>>>>> mail. A
> > > > > > >>>>>>>> better formatted version is
available here:
> > > > > > >>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://docs.google.com/document/d/1VC8lCeErx9kI5lCMPiUn625PO0rxR-iKlVqtt3hkVnk
> > > > > > >>>>>>>> Sorry for that.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Regards,
> > > > > > >>>>>>>> Gábor
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> On 9 March 2016 at 15:51, Gábor
Horváth <
> > > xazax.hun@gmail.com>
> > > > > > >>>> wrote:
> > > > > > >>>>>>>>
> > > > > > >>>>>>>>> Hi,I did not want to send
this proposal out before the
> I
> > > have
> > > > > > some
> > > > > > >>>>>>>>> initial benchmarks, but
this issue was mentioned on the
> > > > mailing
> > > > > > >>>> list
> > > > > > >>>>>> (
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/Tuple-performance-and-the-curious-JIT-compiler-td10666.html
> > > > > > >>>>>>> ),
> > > > > > >>>>>>>>> and I wanted to make this
information available to be
> > able
> > > to
> > > > > > >>>>>>> incorporate
> > > > > > >>>>>>>>> this into that discussion.
I have written this draft
> with
> > > the
> > > > > > help
> > > > > > >>>> of
> > > > > > >>>>>>> Gábor
> > > > > > >>>>>>>>> Gévay and Márton Balassi
and I am open to every
> > suggestion.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> The proposal draft:
> > > > > > >>>>>>>>> Code Generation in Serializers
and Comparators of
> Apache
> > > > Flink
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> I am doing my last semester
of my MSc studies and I’m a
> > > > former
> > > > > > GSoC
> > > > > > >>>>>>>>> student in the LLVM project.
I plan to improve the
> > > > > serialization
> > > > > > >>>>>> code in
> > > > > > >>>>>>>>> Flink during this summer.
The current implementation of
> > the
> > > > > > >>>>>> serializers
> > > > > > >>>>>>> can
> > > > > > >>>>>>>>> be a performance bottleneck
in some scenarios. These
> > > > > performance
> > > > > > >>>>>>> problems
> > > > > > >>>>>>>>> were also reported on the
mailing list recently [1]. I
> > plan
> > > > to
> > > > > > >>>>>> implement
> > > > > > >>>>>>>>> code generation into the
serializers to improve the
> > > > performance
> > > > > > (as
> > > > > > >>>>>>> Stephan
> > > > > > >>>>>>>>> Ewen also suggested.)
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> TODO: I plan to include
some preliminary benchmarks in
> > this
> > > > > > >>>> section.
> > > > > > >>>>>>>>> Performance problems with
the current serializers
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  1.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  PojoSerializer uses reflection
for accessing the
> fields,
> > > > which
> > > > > > >>>> is
> > > > > > >>>>>>>>>  slow (eg. [2])
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  -
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  This is also a serious
problem for the comparators
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  1.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  When deserializing fields
of primitive types (eg.
> int),
> > > the
> > > > > > >>>>>> reusing
> > > > > > >>>>>>>>>  overload of the corresponding
field serializers cannot
> > > > really
> > > > > do
> > > > > > >>>>>> any
> > > > > > >>>>>>> reuse,
> > > > > > >>>>>>>>>  because boxed primitive
types are immutable in Java.
> > This
> > > > > > >>>> results
> > > > > > >>>>>> in
> > > > > > >>>>>>> lots
> > > > > > >>>>>>>>>  of object creations. [3][7]
> > > > > > >>>>>>>>>  2.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  The loop to call the field
serializers makes virtual
> > > > function
> > > > > > >>>>>> calls,
> > > > > > >>>>>>>>>  that cannot be speculatively
devirtualized by the JVM
> or
> > > > > > >>>> predicted
> > > > > > >>>>>>> by the
> > > > > > >>>>>>>>>  CPU, because different
serializer subclasses are
> invoked
> > > for
> > > > > the
> > > > > > >>>>>>> different
> > > > > > >>>>>>>>>  fields. (And the loop
cannot be unrolled, because the
> > > number
> > > > > of
> > > > > > >>>>>>> iterations
> > > > > > >>>>>>>>>  is not a compile time
constant.) See also the
> following
> > > > > > >>>> discussion
> > > > > > >>>>>>> on the
> > > > > > >>>>>>>>>  mailing list [1].
> > > > > > >>>>>>>>>  3.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  A POJO field can have
the value null, so the
> serializer
> > > > > inserts
> > > > > > >>>> 1
> > > > > > >>>>>>>>>  byte null tags, which
wastes space. (Also, the type
> > > > extractor
> > > > > > >>>>>> logic
> > > > > > >>>>>>> does
> > > > > > >>>>>>>>>  not distinguish between
primitive types and their
> boxed
> > > > > > >>>> versions,
> > > > > > >>>>>> so
> > > > > > >>>>>>> even
> > > > > > >>>>>>>>>  an int field has a null
tag.)
> > > > > > >>>>>>>>>  4.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  Subclass tags also add
a byte at the beginning of
> every
> > > POJO
> > > > > > >>>>>>>>>  5.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  getLength() does not know
the size in most cases [4]
> > > > > > >>>>>>>>>  Knowing the size of a
type when serialized has
> numerous
> > > > > > >>>>>> performance
> > > > > > >>>>>>>>>  benefits throughout Flink:
> > > > > > >>>>>>>>>  1.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>     Sorters can do in-place,
when the type is small [5]
> > > > > > >>>>>>>>>     2.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>     Chaining hash tables
do not need resizes, because
> > they
> > > > know
> > > > > > >>>> how
> > > > > > >>>>>>>>>     many buckets to allocate
upfront [6]
> > > > > > >>>>>>>>>     3.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>     Different hash table
architectures could be used,
> eg.
> > > > open
> > > > > > >>>>>>>>>     addressing with linear
probing instead of some
> > chaining
> > > > > > >>>>>>>>>     4.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>     It is possible to deserialize,
modify, and then
> > > serialize
> > > > > > >>>> back
> > > > > > >>>>>> a
> > > > > > >>>>>>>>>     record to its original
place, because it cannot
> > happen
> > > > that
> > > > > > >>>> the
> > > > > > >>>>>>> modified
> > > > > > >>>>>>>>>     version does not fit
in the place allocated there
> for
> > > the
> > > > > old
> > > > > > >>>>>>> version (see
> > > > > > >>>>>>>>>     CompactingHashTable
and ReduceHashTable for
> concrete
> > > > > > >>>> instances
> > > > > > >>>>>> of
> > > > > > >>>>>>> this
> > > > > > >>>>>>>>>     problem)
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> Note, that 2. and 3. are
problems with not just the
> > > > > > PojoSerializer,
> > > > > > >>>>>> but
> > > > > > >>>>>>>>> also with the TupleSerializer.
> > > > > > >>>>>>>>> Solution approaches
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  1.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  Run time code generation
for every POJO
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  -
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>     1. and 3 . would be
automatically solved, if the
> > > > > serializers
> > > > > > >>>>>> for
> > > > > > >>>>>>>>>     POJOs would be generated
on-the-fly (by, for
> example,
> > > > > > >>>>>> Javassist)
> > > > > > >>>>>>>>>     -
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>     2. also needs code
generation, and also some extra
> > > effort
> > > > > in
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>     type extractor to distinguish
between primitive
> types
> > > and
> > > > > > >>>> their
> > > > > > >>>>>>> boxed
> > > > > > >>>>>>>>>     versions
> > > > > > >>>>>>>>>     -
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>     could be used for PojoComparator
as well (which
> could
> > > > > greatly
> > > > > > >>>>>>>>>     increase the performance
of sorting)
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  1.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  Annotations on POJOs (by
the users)
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>  -
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>     Concretely:
> > > > > > >>>>>>>>>     -
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>        annotate fields
that will never be nulls -> no
> > null
> > > > tag
> > > > > > >>>>>> needed
> > > > > > >>>>>>>>>        before every field!
> > > > > > >>>>>>>>>        -
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>        make a POJO final
-> no subclass tag needed
> > > > > > >>>>>>>>>        -
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>        annotating a POJO
that it will not be null -> no
> > top
> > > > > level
> > > > > > >>>>>> null
> > > > > > >>>>>>>>>        tag needed
> > > > > > >>>>>>>>>        -
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>     These would also help
with the getLength problem
> > (6.),
> > > > > > >>>> because
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>     length is often not
known because currently
> anything
> > > can
> > > > be
> > > > > > >>>>>> null
> > > > > > >>>>>>> or a
> > > > > > >>>>>>>>>     subclass can appear
anywhere
> > > > > > >>>>>>>>>     -
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>     These annotations could
be done without code
> > > generation,
> > > > > but
> > > > > > >>>>>> then
> > > > > > >>>>>>>>>     they would add some
overhead when there are no
> > > > annotations
> > > > > > >>>>>>> present, so this
> > > > > > >>>>>>>>>     would work better together
with the code generation
> > > > > > >>>>>>>>>     -
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>     Tuples would become
a special case of POJOs, where
> > > > nothing
> > > > > > >>>> can
> > > > > > >>>>>> be
> > > > > > >>>>>>>>>     null, and no subclass
can appear, so maybe we could
> > > > > eliminate
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>     TupleSerializer
> > > > > > >>>>>>>>>     -
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>     We could annotate some
internal types in Flink
> > > libraries
> > > > > > >>>> (Gelly
> > > > > > >>>>>>>>>     (Vertex, Edge), FlinkML)
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> TODO: what is the situation
with Scala case classes?
> Run
> > > time
> > > > > > code
> > > > > > >>>>>>>>> generation is probably
easier in Scala? (with
> > quasiquotes)
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> About me
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> I am in the last year of
my Computer Science MSc
> studies
> > at
> > > > > > Eotvos
> > > > > > >>>>>>> Lorand
> > > > > > >>>>>>>>> University in Budapest,
and planning to start a PhD in
> > the
> > > > > > autumn.
> > > > > > >>>> I
> > > > > > >>>>>>> have
> > > > > > >>>>>>>>> been working for almost
three years at Ericsson on
> static
> > > > > > analysis
> > > > > > >>>>>> tools
> > > > > > >>>>>>>>> for C++. In 2014 I participated
in GSoC, working on the
> > > LLVM
> > > > > > >>>> project,
> > > > > > >>>>>>> and I
> > > > > > >>>>>>>>> am a frequent contributor
ever since. The next summer I
> > was
> > > > > > >>>>>> interning at
> > > > > > >>>>>>>>> Apple.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> I learned about the Flink
project not too long ago and
> I
> > > like
> > > > > it
> > > > > > so
> > > > > > >>>>>> far.
> > > > > > >>>>>>>>> The last few weeks I was
working on some tickets to
> > > > familiarize
> > > > > > >>>>>> myself
> > > > > > >>>>>>> with
> > > > > > >>>>>>>>> the codebase:
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> https://issues.apache.org/jira/browse/FLINK-3422
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> https://issues.apache.org/jira/browse/FLINK-3322
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> https://issues.apache.org/jira/browse/FLINK-3457
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> My CV is available here:
> > > > > > http://xazax.web.elte.hu/files/resume.pdf
> > > > > > >>>>>>>>> References
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> [1]
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/Tuple-performance-and-the-curious-JIT-compiler-td10666.html
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> [2]
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java#L369
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> [3]
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntSerializer.java#L73
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> [4]
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java#L98
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> [5]
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> [6]
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java#L861
> > > > > > >>>>>>>>> [7] https://issues.apache.org/jira/browse/FLINK-3277
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> Best Regards,
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> Gábor
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>>>>
> > > > > > >>>>
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message