[
https://issues.apache.org/jira/browse/BEAM-5092?focusedWorklogId=137611&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-137611
]
ASF GitHub Bot logged work on BEAM-5092:
----------------------------------------
Author: ASF GitHub Bot
Created on: 23/Aug/18 23:01
Start Date: 23/Aug/18 23:01
Worklog Time Spent: 10m
Work Description: apilloud commented on a change in pull request #6268:
[BEAM-5092] Prevent hash-lookup of schema on every record
URL: https://github.com/apache/beam/pull/6268#discussion_r212480853
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/GetterBasedSchemaProvider.java
##########
@@ -54,10 +55,32 @@
// why is that Java reflection does not guarantee the order in which it
returns fields and
// methods, and these schemas are often based on reflective analysis of
classes. Therefore it's
// important to capture the schema once here, so all invocations of the
toRowFunction see the
- // same version of the schema. If schemaFor were to be called inside the
function, different
+ // same version of the schema. If schemaFor were to be called inside the
lambda below, different
// workers would see different versions of the schema.
Schema schema = schemaFor(typeDescriptor);
- return o ->
Row.withSchema(schema).withFieldValueGetters(fieldValueGetterFactory(),
o).build();
+
+ // Since we know that this factory is always called from inside the lambda
with the same schema,
+ // return a caching factory that caches the first value seen for each
class. This prevents
+ // having to lookup the getter list each time createGetters is called.
+ FieldValueGetterFactory getterFactory =
+ new FieldValueGetterFactory() {
+ transient HashMap<Class, List<FieldValueGetter>> gettersMap;
+
+ @Override
+ public List<FieldValueGetter> createGetters(Class<?> targetClass,
Schema schema) {
+ if (gettersMap == null) {
+ gettersMap = new HashMap<>();
Review comment:
This approach isn't thread safe. Seems like that is a problem, but I'm not
quite sure of the lifecycle here. Should this be a `ConcurrentHashMap` created
along with the getterFactory?
----------------------------------------------------------------
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: 137611)
Time Spent: 8h 10m (was: 8h)
> Nexmark 10x performance regression
> ----------------------------------
>
> Key: BEAM-5092
> URL: https://issues.apache.org/jira/browse/BEAM-5092
> Project: Beam
> Issue Type: New Feature
> Components: sdk-java-core
> Reporter: Andrew Pilloud
> Assignee: Reuven Lax
> Priority: Critical
> Time Spent: 8h 10m
> Remaining Estimate: 0h
>
> There looks to be a 10x performance hit on the DirectRunner and Flink nexmark
> jobs. It first showed up in this build:
> [https://builds.apache.org/view/A-D/view/Beam/job/beam_PostCommit_Java_Nexmark_Direct/151/changes]
> [https://apache-beam-testing.appspot.com/explore?dashboard=5084698770407424]
> [https://apache-beam-testing.appspot.com/explore?dashboard=5699257587728384]
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)