[
https://issues.apache.org/jira/browse/BEAM-13416?focusedWorklogId=746832&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-746832
]
ASF GitHub Bot logged work on BEAM-13416:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 23/Mar/22 19:43
Start Date: 23/Mar/22 19:43
Worklog Time Spent: 10m
Work Description: TheNeuralBit commented on a change in pull request
#16947:
URL: https://github.com/apache/beam/pull/16947#discussion_r833660795
##########
File path:
sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/values/AwsPojoRow.java
##########
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.values;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static
org.apache.beam.sdk.io.aws2.schemas.AwsPojoSchema.CACHE_IDX_OPTION;
+import static
org.apache.beam.sdk.io.aws2.schemas.AwsPojoSchema.CACHE_SIZE_OPTION;
+import static software.amazon.awssdk.core.protocol.MarshallingType.LIST;
+import static software.amazon.awssdk.core.protocol.MarshallingType.MAP;
+import static software.amazon.awssdk.core.protocol.MarshallingType.SDK_POJO;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.function.Function;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.commons.lang3.ArrayUtils;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import software.amazon.awssdk.core.SdkField;
+import software.amazon.awssdk.core.SdkPojo;
+import software.amazon.awssdk.core.protocol.MarshallingType;
+import software.amazon.awssdk.core.traits.ListTrait;
+import software.amazon.awssdk.core.traits.MapTrait;
+
+/**
+ * A row delegating to an underlying AWS {@link SdkPojo}.
+ *
+ * <p>This is following the example of {@link RowWithGetters}, but optimized
for usage with {@link
+ * SdkPojo}s.
Review comment:
Thank you this is great!
I think it's fine to build custom SchemaProviders (rather than
`JavaFieldSchema` and `JavaBeanSchema`) for the AWS types. It's a common
accepted pattern to build SchemaProviders for different domains (e.g. the
protobuf type SchemaProvider in that protobuf extensions). It would be nice if
an existing one "just worked", but the bar for creating a new one should be low
IMO, as long as we take care to re-use things when appropriate.
> Some SDK models are self recursive, these are a major annoyance and
required logic to eventually skip nested recursive fields as Beam schemas don't
support this.
Is it OK to just skip those recursive fields?
> I faced a mismatch when using logical types...
Hm yeah this is a frustrating issue that's come up in several other
contexts. Let's continue discussing it on the [source
PR](https://github.com/apache/beam/pull/11074#discussion_r818731944). It would
be nice to find a solution that's generally applicable.
> The getter factory `Factory<List<FieldValueGetter>>` doesn't work well
with AWS model classes. Model instances already contain the necessary metadata
...
I'm not quite clear on this one, I guess since I have limited context on AWS
model classes vs. Model instances. Is there some reference you could link for
this?
> Additionally, there's a couple of optimisations. RowWithGetters might also
benefit from these:
Could we separate out these changes and apply them to `RowWithGetters` in a
way that's re-usable here?
--
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: 746832)
Time Spent: 4h (was: 3h 50m)
> SqsIO.write input / construction of Sqs request
> -----------------------------------------------
>
> Key: BEAM-13416
> URL: https://issues.apache.org/jira/browse/BEAM-13416
> Project: Beam
> Issue Type: Improvement
> Components: io-java-aws
> Reporter: Moritz Mack
> Priority: P2
> Labels: aws, aws-sdk-v2, sqs
> Time Spent: 4h
> Remaining Estimate: 0h
>
> SqsIO.write should support arbitrary types T and be implemented in terms of a
> mapper function that converts T into an AWS request.
> The current implementation of write as
> {{PTransform<PCollection<SendMessageRequest>, PDone>}} is a rather poor
> choice. It may requires an additional unnecessary serialization &
> deserialisation round. But even worse, the current
> {{SendMessageRequestCoder}} does not support any advanced configuration
> beyond queueUrl and messageBody and will silently drop it without even
> warning the user.
--
This message was sent by Atlassian Jira
(v8.20.1#820001)