[ 
https://issues.apache.org/jira/browse/BEAM-6276?focusedWorklogId=178336&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-178336
 ]

ASF GitHub Bot logged work on BEAM-6276:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 23/Dec/18 15:16
            Start Date: 23/Dec/18 15:16
    Worklog Time Spent: 10m 
      Work Description: reuvenlax commented on issue #7331: [BEAM-6276] Fix 
performance regression.
URL: https://github.com/apache/beam/pull/7331#issuecomment-449643044
 
 
   BTW one thing that I suspect would make things faster:
   
   Today SchemaCoder decodes into a Row object (essentially a hashmap), and we
   then have to construct the user object from this Row. I do wonder if it
   would be possible to decode directly into constructor calls for the user
   object, and skip constructing the Row first. The risk is that it might
   complicate the code quite a bit by forcing SchemaCoder to know about things
   it doesn't today. It also might not have as much of a benefit for real
   pipelines where encoding/decoding is less frequent than other operations
   (as runners tend to fuse multiple operations together).
   
   On Sun, Dec 23, 2018 at 7:08 AM Reuven Lax <[email protected]> wrote:
   
   > I'm not sure why deterministic schemas are needed here. We already
   > generate such constructors for POJOs, and if you look at the PR adding
   > AutoValue support (https://github.com/apache/beam/pull/7334 - includes
   > commits from other PRs, so I would look at just
   > AutoValueUtils$BuilderCreateInstruction), we do that there for AutoValues.
   > It requires a bit of extra code complexity to map arguments to constructor
   > arguments, but since the schema is deterministically known at the time the
   > byte-buddy code is generated we can do it then and not have to do it again.
   >
   > In general, ensuring deterministic schema order will be tricky. We want to
   > support updating schemas - i.e. adding new fields to a schema - in the
   > future. To do this in a compatible way (without forcing a user to drain and
   > restart the pipeline) means that we will need to maintain the old indices
   > so the old encoded values can still be read. This means that a simple
   > lexicographical ordering won't work (otherwise new fields might end up at
   > the beginning). It's maybe still possible if you kept track of a pipeline
   > version and integrated that into the ordering, but I think that going that
   > route might get very complicated.
   >
   > On Sun, Dec 23, 2018 at 3:28 AM Gleb Kanterov <[email protected]>
   > wrote:
   >
   >> @reuvenlax <https://github.com/reuvenlax> agree that it isn't a problem
   >> for graph-construction. I didn't elaborate properly, but the reason to 
have
   >> deterministic schema is to pass it to a method like:
   >>
   >> class FromRowUsingCreatorGenerator {
   >>   public static <T> FromRowUsingCreator<T> generate(Class<T> clazz, 
Schema schema);
   >> }
   >>
   >> and then generate byte code based on a schema and class:
   >>
   >> public class GeneratedSchemaUserTypeCreator extends 
SerializableFunction4<Object, Object, Object, Object, JavaBean> implements 
UserTypeCreatorFactory {
   >>   private final FieldValueSetter[] setters;
   >>
   >>   Object apply(Object... args) { // for UserTypeCreatorFactory
   >>     return apply(args[0], args[1], args[2], args[3]);
   >>   }
   >>
   >>   Object apply(Object p0, Object p1, Object p2, Object p3) { // faster
   >>         // we don't use newInstance, instead just generate byte-code with 
a constructor call
   >>         Object object = new JBean();
   >>         setters[0].set(object, p0);
   >>         setters[1].set(object, p1);
   >>         setters[2].set(object, p2);
   >>         setters[3].set(object, p3);
   >>   }
   >> }
   >> public class GeneratedFromRowUsingCreator extends 
FromRowUsingCreator<JavaBean> {
   >>
   >>   private final SerializableFunction4 creator;
   >>   private final FromRowUsingCreator<InnerJavaBean> underlying1; // for 
field_1
   >>
   >>   Generated(Schema schema) {
   >>     // know that it is SerializableFunction4 because there are 4 fields 
in schema
   >>     creator = (SerializableFunction4) 
schemaTypeCreatorFactory.create(JavaBean.class, schema);
   >>   }
   >>
   >>   public T apply(Row row) {
   >>     // calling .apply(Object p0, ..., Object p3) is much faster then 
`.apply(Object... params)`
   >>     // due to JIT
   >>     return creator.apply(
   >>         row.getValue(0),
   >>         // byte code will contain a call to underling FromRowUsingCreator
   >>         // only in the case of ROW, MAP or ARRAY field
   >>         underlying.toRow(row.getRow(1)),
   >>         row.getValue(2),
   >>         row.getValue(3));
   >>   }
   >>
   >> }
   >>
   >> When I did benchmark, I used JMH, and didn't include any cost that we pay
   >> once per pipeline construction.
   >>
   >> —
   >> You are receiving this because you were mentioned.
   >> Reply to this email directly, view it on GitHub
   >> <https://github.com/apache/beam/pull/7331#issuecomment-449630174>, or mute
   >> the thread
   >> 
<https://github.com/notifications/unsubscribe-auth/AUGE1R4rxmjlQZdRud8MkfiV1e1zuvUgks5u72jtgaJpZM4ZbvGg>
   >> .
   >>
   >
   
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 178336)
    Time Spent: 3h  (was: 2h 50m)

> Performance regression caused by extra calls to TypeDescriptor.getRawType
> -------------------------------------------------------------------------
>
>                 Key: BEAM-6276
>                 URL: https://issues.apache.org/jira/browse/BEAM-6276
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-java-core
>            Reporter: Reuven Lax
>            Assignee: Reuven Lax
>            Priority: Major
>          Time Spent: 3h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to