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

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

                Author: ASF GitHub Bot
            Created on: 10/Mar/20 17:35
            Start Date: 10/Mar/20 17:35
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on pull request #9758: [BEAM-8374] 
Enable returning missing PublishResult fields in SnsIO.Write
URL: https://github.com/apache/beam/pull/9758#discussion_r390477368
 
 

 ##########
 File path: 
sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/coders/AwsCoders.java
 ##########
 @@ -0,0 +1,156 @@
+/*
+ * 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.aws.coders;
+
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.http.HttpResponse;
+import com.amazonaws.http.SdkHttpMetadata;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+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 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/** {@link Coder}s for common AWS SDK objects. */
+public final class AwsCoders {
+
+  private AwsCoders() {}
+
+  /**
+   * Returns a new coder for ResponseMetadata.
+   *
+   * @return the ResponseMetadata coder
+   */
+  public static Coder<ResponseMetadata> responseMetadata() {
+    return ResponseMetadataCoder.of();
+  }
+
+  /**
+   * Returns a new coder for SdkHttpMetadata that by default does not 
serialize the response
+   * headers.
+   *
+   * @return the SdkHttpMetadata coder
+   */
+  public static Coder<SdkHttpMetadata> sdkHttpMetadata() {
+    return new SdkHttpMetadataCoder(false);
+  }
+
+  /**
+   * Returns a new coder for SdkHttpMetadata that will serialize the response 
headers if indicated.
+   *
+   * @param includeHeaders <code>true</code> to serialize the SDK response 
headers, otherwise <code>
+   *     false</code>
+   * @return the SdkHttpMetadata coder
+   */
+  public static Coder<SdkHttpMetadata> sdkHttpMetadata(boolean includeHeaders) 
{
+    return new SdkHttpMetadataCoder(includeHeaders);
+  }
+
+  private static class ResponseMetadataCoder extends Coder<ResponseMetadata>
+      implements Serializable {
+
+    private static final Coder<Map<String, String>> METADATA_ENCODER =
+        NullableCoder.of(MapCoder.of(StringUtf8Coder.of(), 
StringUtf8Coder.of()));
+    private static final ResponseMetadataCoder INSTANCE = new 
ResponseMetadataCoder();
+
+    private ResponseMetadataCoder() {}
+
+    public static ResponseMetadataCoder of() {
+      return INSTANCE;
+    }
+
+    @Override
+    public void encode(ResponseMetadata value, OutputStream outStream)
+        throws CoderException, IOException {
+      METADATA_ENCODER.encode(
+          ImmutableMap.of(ResponseMetadata.AWS_REQUEST_ID, 
value.getRequestId()), outStream);
+    }
+
+    @Override
+    public ResponseMetadata decode(InputStream inStream) throws 
CoderException, IOException {
+      return new ResponseMetadata(METADATA_ENCODER.decode(inStream));
+    }
+
+    @Override
+    public List<? extends Coder<?>> getCoderArguments() {
+      return ImmutableList.of();
+    }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {
+      METADATA_ENCODER.verifyDeterministic();
+    }
+  }
+
+  private static class SdkHttpMetadataCoder extends Coder<SdkHttpMetadata> 
implements Serializable {
+
+    private static final Coder<Integer> STATUS_CODE_CODER = VarIntCoder.of();
+    private static final Coder<Map<String, String>> HEADERS_ENCODER =
+        NullableCoder.of(MapCoder.of(StringUtf8Coder.of(), 
StringUtf8Coder.of()));
+
+    private final boolean includeHeaders;
+
+    protected SdkHttpMetadataCoder(boolean includeHeaders) {
+      this.includeHeaders = includeHeaders;
+    }
+
+    @Override
+    public void encode(SdkHttpMetadata value, OutputStream outStream)
+        throws CoderException, IOException {
+      STATUS_CODE_CODER.encode(value.getHttpStatusCode(), outStream);
+      if (includeHeaders) {
+        HEADERS_ENCODER.encode(value.getHttpHeaders(), outStream);
+      }
+    }
+
+    @Override
+    public SdkHttpMetadata decode(InputStream inStream) throws CoderException, 
IOException {
+      final int httpStatusCode = STATUS_CODE_CODER.decode(inStream);
+      HttpResponse httpResponse = new HttpResponse(null, null);
+      httpResponse.setStatusCode(httpStatusCode);
+      if (includeHeaders) {
+        Optional.ofNullable(HEADERS_ENCODER.decode(inStream))
+            .ifPresent(
+                headers ->
+                    headers.keySet().forEach(k -> httpResponse.addHeader(k, 
headers.get(k))));
+      }
+      return SdkHttpMetadata.from(httpResponse);
+    }
+
+    @Override
+    public List<? extends Coder<?>> getCoderArguments() {
+      return ImmutableList.of();
+    }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {
+      STATUS_CODE_CODER.verifyDeterministic();
+      HEADERS_ENCODER.verifyDeterministic();
 
 Review comment:
   ```suggestion
         if (includeHeaders) {
           HEADERS_ENCODER.verifyDeterministic();
         }
   ```
 
----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

    Worklog Id:     (was: 400877)
    Time Spent: 6h  (was: 5h 50m)

> PublishResult returned by SnsIO is missing sdkResponseMetadata and 
> sdkHttpMetadata
> ----------------------------------------------------------------------------------
>
>                 Key: BEAM-8374
>                 URL: https://issues.apache.org/jira/browse/BEAM-8374
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-aws
>    Affects Versions: 2.13.0, 2.14.0, 2.15.0
>            Reporter: Jonothan Farr
>            Assignee: Jonothan Farr
>            Priority: Minor
>          Time Spent: 6h
>  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.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to