You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2022/01/06 18:12:00 UTC

[jira] [Work logged] (BEAM-13243) PublishResult returned by SnsIO is missing sdkResponseMetadata and sdkHttpMetadata (AWS SDK v2)

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

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

                Author: ASF GitHub Bot
            Created on: 06/Jan/22 18:11
            Start Date: 06/Jan/22 18:11
    Worklog Time Spent: 10m 
      Work Description: aromanenko-dev commented on a change in pull request #16128:
URL: https://github.com/apache/beam/pull/16128#discussion_r779735655



##########
File path: sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/coders/AwsCoders.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.io.aws2.coders;
+
+import static software.amazon.awssdk.awscore.util.AwsHeader.AWS_REQUEST_ID;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import software.amazon.awssdk.awscore.AwsResponseMetadata;
+import software.amazon.awssdk.http.SdkHttpResponse;
+import software.amazon.awssdk.utils.ImmutableMap;
+
+/** {@link Coder}s for common AWS SDK objects. */
+public final class AwsCoders {

Review comment:
       Do we really need to have this class and its methods public?

##########
File path: sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/coders/AwsCoders.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.io.aws2.coders;
+
+import static software.amazon.awssdk.awscore.util.AwsHeader.AWS_REQUEST_ID;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import software.amazon.awssdk.awscore.AwsResponseMetadata;
+import software.amazon.awssdk.http.SdkHttpResponse;
+import software.amazon.awssdk.utils.ImmutableMap;
+
+/** {@link Coder}s for common AWS SDK objects. */
+public final class AwsCoders {
+
+  private AwsCoders() {}
+
+  /**
+   * Returns a new coder for {@link AwsResponseMetadata} (AWS request ID only).
+   *
+   * @return the {@link AwsResponseMetadata} coder
+   */
+  public static Coder<AwsResponseMetadata> awsResponseMetadata() {

Review comment:
       Would  it be better to use a factory method here to create different coders? Or even more - just delegate it to `of()` methods for every coder as we used to have for other coders in Beam?

##########
File path: sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/PublishResponseCoders.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.io.aws2.sns;
+
+import static org.apache.beam.sdk.io.aws2.coders.AwsCoders.sdkHttpResponse;
+import static org.apache.beam.sdk.io.aws2.coders.AwsCoders.sdkHttpResponseWithoutHeaders;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.aws2.coders.AwsCoders;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import software.amazon.awssdk.awscore.AwsResponseMetadata;
+import software.amazon.awssdk.http.SdkHttpResponse;
+import software.amazon.awssdk.services.sns.model.PublishResponse;
+
+/** Coders for SNS {@link PublishResponse}. */
+public class PublishResponseCoders {
+  private static final Coder<String> MESSAGE_ID_CODER = StringUtf8Coder.of();
+  private static final NullableCoder<AwsResponseMetadata> METADATA_CODER =
+      NullableCoder.of(AwsCoders.awsResponseMetadata());
+
+  private PublishResponseCoders() {}
+
+  /**
+   * Returns a new SNS {@link PublishResponse} coder which by default serializes only the SNS
+   * messageId.
+   *
+   * @return the {@link PublishResponse} coder
+   */
+  public static Coder<PublishResponse> defaultPublishResponse() {

Review comment:
       The general naming pattern for coder instantiating method is `YourCoder of(...)`. So why not to follow the same pattern and to add such method(s) right into the coder class, like all other Beam coders do and it was before, and avoid wrapper classes ?

##########
File path: sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java
##########
@@ -335,6 +347,27 @@ private static boolean isTopicExists(SnsClient client, String topicArn) {
       }
     }
 
+    /**
+     * Encode the full {@code PublishResult} object, including sdkResponseMetadata and
+     * sdkHttpMetadata with the HTTP response headers.
+     */
+    public Write<T> withFullPublishResponse() {

Review comment:
       Why not to make it by default? Too big result overhead?

##########
File path: sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java
##########
@@ -335,6 +347,27 @@ private static boolean isTopicExists(SnsClient client, String topicArn) {
       }
     }
 
+    /**
+     * Encode the full {@code PublishResult} object, including sdkResponseMetadata and
+     * sdkHttpMetadata with the HTTP response headers.
+     */
+    public Write<T> withFullPublishResponse() {
+      return withCoder(PublishResponseCoders.fullPublishResponse());
+    }
+
+    /**
+     * Encode the full {@code PublishResult} object, including sdkResponseMetadata and
+     * sdkHttpMetadata but excluding the HTTP response headers.
+     */
+    public Write<T> withFullPublishResponseWithoutHeaders() {
+      return withCoder(PublishResponseCoders.fullPublishResponseWithoutHeaders());
+    }
+
+    /** Encode the {@code PublishResult} with the given coder. */

Review comment:
       Do we really need to provide this method to user?

##########
File path: sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java
##########
@@ -335,6 +347,27 @@ private static boolean isTopicExists(SnsClient client, String topicArn) {
       }
     }
 
+    /**
+     * Encode the full {@code PublishResult} object, including sdkResponseMetadata and
+     * sdkHttpMetadata with the HTTP response headers.
+     */
+    public Write<T> withFullPublishResponse() {
+      return withCoder(PublishResponseCoders.fullPublishResponse());
+    }
+
+    /**
+     * Encode the full {@code PublishResult} object, including sdkResponseMetadata and
+     * sdkHttpMetadata but excluding the HTTP response headers.
+     */
+    public Write<T> withFullPublishResponseWithoutHeaders() {

Review comment:
       nit: `withFullPublishResponse(boolean excludeHeaders)`?




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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

    Worklog Id:     (was: 704681)
    Time Spent: 2h 10m  (was: 2h)

> PublishResult returned by SnsIO is missing sdkResponseMetadata and sdkHttpMetadata (AWS SDK v2)
> -----------------------------------------------------------------------------------------------
>
>                 Key: BEAM-13243
>                 URL: https://issues.apache.org/jira/browse/BEAM-13243
>             Project: Beam
>          Issue Type: Improvement
>          Components: io-java-aws
>            Reporter: Moritz Mack
>            Assignee: Moritz Mack
>            Priority: P3
>              Labels: aws
>          Time Spent: 2h 10m
>  Remaining Estimate: 0h
>
> Currently the PublishResultCoder in SnsIO only serializes the messageId field so the PublishResult returned by Beam returns null for getSdkResponseMetadata() and getSdkHttpMetadata(). This makes it impossible to check the HTTP status for errors, which is necessary since this is not handled in SnsIO.
> Support for this was added in BEAM-8374 for AWS SDK v1, but is still missing for AWS SDK v2.
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)