[
https://issues.apache.org/jira/browse/BEAM-13056?focusedWorklogId=675280&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-675280
]
ASF GitHub Bot logged work on BEAM-13056:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 04/Nov/21 00:36
Start Date: 04/Nov/21 00:36
Worklog Time Spent: 10m
Work Description: reuvenlax commented on a change in pull request #15886:
URL: https://github.com/apache/beam/pull/15886#discussion_r742418785
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
##########
@@ -683,6 +686,18 @@ public static DoFnSchemaInformation
getDoFnSchemaInformation(
(SchemaCoder<?>) input.getCoder(), accessDescriptor,
selectedSchema, elementT);
}
}
+ if (input.hasSchema()) {
Review comment:
unnecessary - we wouldn't hit this codepath if !input.hasSchema
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSchemaInformation.java
##########
@@ -101,7 +107,16 @@ DoFnSchemaInformation withSelectFromSchemaParameter(
unbox))
.build();
- return toBuilder().setElementConverters(converters).build();
+ List<FieldAccessDescriptor> descriptors =
+ ImmutableList.<FieldAccessDescriptor>builder()
+ .addAll(getFieldAccessDescriptors())
+ .add(selectDescriptor)
+ .build();
+
+ return toBuilder()
+ .setElementConverters(converters)
+ .setFieldAccessDescriptors(descriptors)
+ .build();
Review comment:
This is not always called (withUnboxFromPrimitiveParameter as called as
well). I think better to make this a separate builder called from ParDo.java
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSchemaInformation.java
##########
@@ -101,7 +107,16 @@ DoFnSchemaInformation withSelectFromSchemaParameter(
unbox))
.build();
- return toBuilder().setElementConverters(converters).build();
+ List<FieldAccessDescriptor> descriptors =
+ ImmutableList.<FieldAccessDescriptor>builder()
+ .addAll(getFieldAccessDescriptors())
+ .add(selectDescriptor)
+ .build();
Review comment:
Instead of storing a List, just store one FieldAccessDescriptor and use
FieldAccessDescriptor.union to merge them.
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
##########
@@ -683,6 +686,18 @@ public static DoFnSchemaInformation
getDoFnSchemaInformation(
(SchemaCoder<?>) input.getCoder(), accessDescriptor,
selectedSchema, elementT);
}
}
+ if (input.hasSchema()) {
Review comment:
unnecessary - we wouldn't hit this codepath if !input.hasSchema
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSchemaInformation.java
##########
@@ -101,7 +107,16 @@ DoFnSchemaInformation withSelectFromSchemaParameter(
unbox))
.build();
- return toBuilder().setElementConverters(converters).build();
+ List<FieldAccessDescriptor> descriptors =
+ ImmutableList.<FieldAccessDescriptor>builder()
+ .addAll(getFieldAccessDescriptors())
+ .add(selectDescriptor)
+ .build();
+
+ return toBuilder()
+ .setElementConverters(converters)
+ .setFieldAccessDescriptors(descriptors)
+ .build();
Review comment:
This is not always called (withUnboxFromPrimitiveParameter as called as
well). I think better to make this a separate builder called from ParDo.java
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSchemaInformation.java
##########
@@ -101,7 +107,16 @@ DoFnSchemaInformation withSelectFromSchemaParameter(
unbox))
.build();
- return toBuilder().setElementConverters(converters).build();
+ List<FieldAccessDescriptor> descriptors =
+ ImmutableList.<FieldAccessDescriptor>builder()
+ .addAll(getFieldAccessDescriptors())
+ .add(selectDescriptor)
+ .build();
Review comment:
Instead of storing a List, just store one FieldAccessDescriptor and use
FieldAccessDescriptor.union to merge them.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 675280)
Time Spent: 1h (was: 50m)
> Add method to fetch ProcessContext FieldAccess
> ----------------------------------------------
>
> Key: BEAM-13056
> URL: https://issues.apache.org/jira/browse/BEAM-13056
> Project: Beam
> Issue Type: New Feature
> Components: sdk-java-core
> Reporter: Andrew Pilloud
> Assignee: Andrew Pilloud
> Priority: P1
> Time Spent: 1h
> Remaining Estimate: 0h
>
> There is DoFnSignature.fieldAccessDeclarations, which returns declared
> FieldAccess, but we are missing a way to return the set of FieldAccess that
> is actually used.
> Aside from what is returned by fieldAccessDeclarations, there are implicit
> FieldAccessDescriptor.withAllFields() created by @Element and @ProcessContext.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)