jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r398200394
 
 

 ##########
 File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,658 @@
+/*
+ * 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.gcp.healthcare;
+
+import com.google.api.services.healthcare.v1alpha2.model.Message;
+import com.google.auto.value.AutoValue;
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.gcp.datastore.AdaptiveThrottler;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.Sleeper;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link HL7v2IO} provides an API for reading from and writing to <a
+ * href="https://cloud.google.com/healthcare/docs/concepts/hl7v2";>Google Cloud 
Healthcare HL7v2 API.
+ * </a>
+ *
+ * <p>Read
+ *
+ * <p>HL7v2 Messages are fetched from the HL7v2 store based on the {@link 
PCollection} of message
+ * IDs {@link String}s as {@link HL7v2IO.Read.Result} where one can call {@link
+ * Read.Result#getMessages()} to retrived a {@link PCollection} containing the 
successfully fetched
+ * {@link HL7v2Message}s and/or {@link Read.Result#getFailedReads()} to 
retrieve a {@link
+ * PCollection} of {@link HealthcareIOError} containing the msgID that could 
not be fetched and the
+ * exception as a {@link HealthcareIOError<String>}, this can be used to write 
to the dead letter
+ * storage system of your choosing.
+ *
+ * <p>Write
+ *
+ * <p>A bounded or unbounded {@link PCollection} of {@link HL7v2Message} can 
be ingested into an
+ * HL7v2 store using {@link HL7v2IO#ingestMessages(String)}. This will return 
a {@link
+ * HL7v2IO.Write.Result} on which you can call {@link 
Write.Result#getFailedInsertsWithErr()} to
+ * retrieve a {@link PCollection} of {@link HealthcareIOError<HL7v2Message>} 
containing the Message
+ * that failed to be ingested and the exception. This can be used to write to 
the dead letter
+ * storage system of your chosing.
+ *
+ * <p>Unbounded Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * HL7v2IO.Read.Result readResult = p
+ *   .apply(
+ *     "Read HL7v2 notifications",
+ *     PubSubIO.readStrings().fromTopic(options.getNotificationSubscription()))
+ *   .apply(HL7v2IO.readAll());
+ *
+ * // Write errors to your favorite dead letter  queue (e.g. Pub/Sub, GCS, 
BigQuery)
+ * readResult.getFailedReads().apply("WriteToDeadLetterQueue", ...);
+ *
+ *
+ * // Go about your happy path transformations.
+ * PCollection<HL7v2Message> out = 
readResult.getMessages().apply("ProcessFetchedMessages", ...);
+ *
+ * // Write using the Message.Ingest method of the HL7v2 REST API.
+ * out.apply(HL7v2IO.ingestMessages(options.getOutputHL7v2Store()));
+ *
+ * pipeline.run();
+ *
+ * }***
+ * </pre>
+ *
+ * <p>Bounded Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * HL7v2IO.Read.Result readResult = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2MessageIDs(
+ *           Collections.singletonList(options.getInputHL7v2Store()), 
option.getHL7v2Filter()))
+ *   .apply(HL7v2IO.readAll());
+ *
+ * // Write errors to your favorite dead letter  queue (e.g. Pub/Sub, GCS, 
BigQuery)
+ * readResult.getFailedReads().apply("WriteToDeadLetterQueue", ...);
+ *
+ *
+ * // Go about your happy path transformations.
+ * PCollection<HL7v2Message> out = 
readResult.getMessages().apply("ProcessFetchedMessages", ...);
+ *
+ * // Write using the Message.Ingest method of the HL7v2 REST API.
+ * out.apply(HL7v2IO.ingestMessages(options.getOutputHL7v2Store()));
+ *
+ * pipeline.run().waitUntilFinish();
+ * }***
+ * </pre>
+ */
+public class HL7v2IO {
+
+  private static Write.Builder write(String hl7v2Store) {
+    return new AutoValue_HL7v2IO_Write.Builder().setHL7v2Store(hl7v2Store);
+  }
+
+  public static Read readAll() {
+    return new Read();
+  }
+
+  /**
+   * Write with Messages.Ingest method. @see <a
+   * 
href=https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/ingest></a>
+   *
+   * @param hl7v2Store the hl 7 v 2 store
+   * @return the write
+   */
+  public static Write ingestMessages(String hl7v2Store) {
+    return write(hl7v2Store).setWriteMethod(Write.WriteMethod.INGEST).build();
+  }
+
+  // TODO add hyper links to this doc string.
+  /**
+   * The type Read that reads HL7v2 message contents given a PCollection of 
message IDs strings.
+   *
+   * <p>These could be sourced from any {@link PCollection} of {@link String}s 
but the most popular
+   * patterns would be {@link PubsubIO#readStrings()} reading a subscription 
on an HL7v2 Store's
+   * notification channel topic or using {@link ListHL7v2MessageIDs} to list 
HL7v2 message IDs with
+   * an optional filter using Ingest write method. @see <a
+   * 
href=https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list></a>.
+   */
+  public static class Read extends PTransform<PCollection<String>, 
Read.Result> {
+
+    public Read() {}
+
+    public static class Result implements POutput, PInput {
+      private PCollection<HL7v2Message> messages;
+
+      private PCollection<HealthcareIOError<String>> failedReads;
+      PCollectionTuple pct;
+
+      public static Result of(PCollectionTuple pct) throws 
IllegalArgumentException {
+        if (pct.getAll()
+            .keySet()
+            .containsAll((Collection<?>) 
TupleTagList.of(OUT).and(DEAD_LETTER))) {
+          return new Result(pct);
+        } else {
+          throw new IllegalArgumentException(
+              "The PCollection tuple must have the HL7v2IO.Read.OUT "
+                  + "and HL7v2IO.Read.DEAD_LETTER tuple tags");
+        }
+      }
+
+      private Result(PCollectionTuple pct) {
+        this.pct = pct;
+        this.messages = pct.get(OUT).setCoder(new HL7v2MessageCoder());
+        this.failedReads =
+            pct.get(DEAD_LETTER).setCoder(new 
HealthcareIOErrorCoder<>(StringUtf8Coder.of()));
+      }
+
+      public PCollection<HealthcareIOError<String>> getFailedReads() {
+        return failedReads;
+      }
+
+      public PCollection<HL7v2Message> getMessages() {
+        return messages;
+      }
+
+      @Override
+      public Pipeline getPipeline() {
+        return this.pct.getPipeline();
+      }
+
+      @Override
+      public Map<TupleTag<?>, PValue> expand() {
+        return ImmutableMap.of(OUT, messages);
+      }
+
+      @Override
+      public void finishSpecifyingOutput(
+          String transformName, PInput input, PTransform<?, ?> transform) {}
+    }
+
+    /** The tag for the main output of HL7v2 Messages. */
+    public static final TupleTag<HL7v2Message> OUT = new 
TupleTag<HL7v2Message>() {};
+    /** The tag for the deadletter output of HL7v2 Messages. */
+    public static final TupleTag<HealthcareIOError<String>> DEAD_LETTER =
+        new TupleTag<HealthcareIOError<String>>() {};
+
+    @Override
+    public Result expand(PCollection<String> input) {
+      return input.apply("Fetch HL7v2 messages", new FetchHL7v2Message());
+    }
+
+    /**
+     * DoFn to fetch a message from an Google Cloud Healthcare HL7v2 store 
based on msgID
+     *
+     * <p>This DoFn consumes a {@link PCollection} of notifications {@link 
String}s from the HL7v2
+     * store, and fetches the actual {@link HL7v2Message} object based on the 
id in the notification
+     * and will output a {@link PCollectionTuple} which contains the output 
and dead-letter {@link
+     * PCollection}.
+     *
+     * <p>The {@link PCollectionTuple} output will contain the following 
{@link PCollection}:
+     *
+     * <ul>
+     *   <li>{@link HL7v2IO.Read#OUT} - Contains all {@link PCollection} 
records successfully read
+     *       from the HL7v2 store.
+     *   <li>{@link HL7v2IO.Read#DEAD_LETTER} - Contains all {@link 
PCollection} of {@link
+     *       HealthcareIOError<String>} message IDs which failed to be fetched 
from the HL7v2 store,
+     *       with error message and stacktrace.
+     * </ul>
+     *
+     * <p>Example:
+     *
+     * <pre>{@code
+     * PipelineOptions options = ...;
+     * Pipeline pipeline = Pipeline.create(options)
+     *
+     * PCollection<String> msgIDs = pipeline.apply(
+     *    "ReadHL7v2Notifications",
+     *    
PubsubIO.readStrings().fromSubscription(options.getInputSubscription()));
+     *
+     * PCollectionTuple fetchResults = msgIDs.apply(
+     *    "FetchHL7v2Messages",
+     *    new FetchHL7v2Message;
+     *
+     * // Write errors to your favorite dead letter  queue (e.g. Pub/Sub, GCS, 
BigQuery)
+     * fetchResults.get(PubsubNotificationToHL7v2Message.DEAD_LETTER)
+     *    .apply("WriteToDeadLetterQueue", ...);
+     *
+     * PCollection<Message> fetchedMessages = 
fetchResults.get(PubsubNotificationToHL7v2Message.OUT)
+     *    .apply("ExtractFetchedMessage",
+     *    MapElements
+     *        .into(TypeDescriptor.of(Message.class))
+     *        .via(FailsafeElement::getPayload));
+     *
+     * // Go about your happy path transformations.
+     * fetchedMessages.apply("ProcessFetchedMessages", ...)
+     *
+     * }****
+     * </pre>
+     */
+    public static class FetchHL7v2Message extends 
PTransform<PCollection<String>, Result> {
+
+      /** Instantiates a new Fetch HL7v2 message DoFn. */
+      public FetchHL7v2Message() {}
+
+      @Override
+      public Result expand(PCollection<String> msgIds) {
+        return new Result(
+            msgIds.apply(
+                ParDo.of(new FetchHL7v2Message.HL7v2MessageGetFn())
+                    .withOutputTags(HL7v2IO.Read.OUT, 
TupleTagList.of(HL7v2IO.Read.DEAD_LETTER))));
+      }
+
+      /** DoFn for fetching messages from the HL7v2 store with error handling. 
*/
+      public static class HL7v2MessageGetFn extends DoFn<String, HL7v2Message> 
{
+
+        private Counter failedMessageGets =
+            Metrics.counter(FetchHL7v2Message.HL7v2MessageGetFn.class, 
"failed-message-reads");
+        private static final Logger LOG =
+            LoggerFactory.getLogger(FetchHL7v2Message.HL7v2MessageGetFn.class);
+        private final Counter throttledSeconds =
+            Metrics.counter(
+                FetchHL7v2Message.HL7v2MessageGetFn.class, 
"cumulative-throttling-seconds");
+        private final Counter successfulHL7v2MessageGets =
+            Metrics.counter(
+                FetchHL7v2Message.HL7v2MessageGetFn.class, 
"successful-hl7v2-message-gets");
+        private HealthcareApiClient client;
+        private transient AdaptiveThrottler throttler;
+
+        /** Instantiates a new Hl 7 v 2 message get fn. */
+        HL7v2MessageGetFn() {}
+
+        /**
+         * Instantiate healthcare client.
+         *
+         * @throws IOException the io exception
+         */
+        @Setup
+        public void instantiateHealthcareClient() throws IOException {
+          this.client = new HttpHealthcareApiClient();
+        }
+
+        /** Start bundle. */
+        @StartBundle
+        public void startBundle() {
+          if (throttler == null) {
+            throttler = new AdaptiveThrottler(1200000, 10000, 1.25);
+          }
+        }
+
+        /**
+         * Process element.
+         *
+         * @param context the context
+         */
+        @ProcessElement
+        public void processElement(ProcessContext context) {
+          String msgId = context.element();
+          try {
+            context.output(HL7v2Message.fromModel(fetchMessage(this.client, 
msgId)));
+          } catch (Exception e) {
+            failedMessageGets.inc();
+            LOG.warn(
+                String.format(
+                    "Error fetching HL7v2 message with ID %s writing to Dead 
Letter "
+                        + "Queue. Cause: %s Stack Trace: %s",
+                    msgId, e.getMessage(), 
Throwables.getStackTraceAsString(e)));
+            context.output(HL7v2IO.Read.DEAD_LETTER, 
HealthcareIOError.of(msgId, e));
+          }
+        }
+
+        private Message fetchMessage(HealthcareApiClient client, String msgId)
+            throws IOException, ParseException, IllegalArgumentException, 
InterruptedException {
+          final int throttleWaitSeconds = 5;
+          long startTime = System.currentTimeMillis();
+          Sleeper sleeper = Sleeper.DEFAULT;
+          if (throttler.throttleRequest(startTime)) {
+            LOG.info(String.format("Delaying request for %s due to previous 
failures.", msgId));
+            this.throttledSeconds.inc(throttleWaitSeconds);
+            sleeper.sleep(throttleWaitSeconds * 1000);
+          }
+
+          com.google.api.services.healthcare.v1alpha2.model.Message msg =
+              client.getHL7v2Message(msgId);
+
+          this.throttler.successfulRequest(startTime);
+          if (msg == null) {
+            throw new IOException(String.format("GET request for %s returned 
null", msgId));
+          }
+          this.successfulHL7v2MessageGets.inc();
+          return msg;
+        }
+      }
+    }
+  }
+
+  /** The type List HL7v2 message IDs. */
+  public static class ListHL7v2MessageIDs extends PTransform<PBegin, 
PCollection<String>> {
+
+    private final List<String> hl7v2Stores;
+    private final String filter;
+
+    /**
+     * Instantiates a new List HL7v2 message IDs with filter.
+     *
+     * @param hl7v2Stores the HL7v2 stores
+     * @param filter the filter
+     */
+    ListHL7v2MessageIDs(List<String> hl7v2Stores, String filter) {
 
 Review comment:
   Thanks for bringing this up.
   Agreed on supporting getting the messages from the alpha2 list API to save 
the extra retrieval.
   
   IMO this DoFn should remain as is to support backwards compatibility with 
the beta1 list api (which only has message IDs in the response) through a 
similar code path to the real time use cases which would subscribe to pubsub 
notifications (which also only have the message IDs and require retrieval).
   
   @pabloem @lastomato thoughts on how we should best handle the two different 
APIs minimizing calls where possible?
   
   I see two possible strategies:
   1. Rely on user to use a different PTransform depending on which API version 
they are trying to hit.
      - For beta1 api requiring "get" after list: `HL7v2IO.ListHL7v2MessageIDs` 
-> `HL7v2IO.Read`(current path).
      - Add a new PTransform for alpha2 api (where message contents are in the 
list): `HL7v2IO.V2.Read extends PTransform<PBegin, HL7v2Message>`
   1. Restructuring so we can support both API responses with a single 
composite PTransform that follows this logic:
      - `ListMessageFn` makes the message list call and outputs to two 
`PCollection`s with the following logic:
           - IF response contains the `hl7V2Messages` key and either output the 
messages to the `PCollection<HL7v2Message>` with a `V2` tuple tag
           - ELSE  output the messageIDs `PCollection<String>` with a `V1` 
tuple tag
      - On the `V1` tagged PCollection apply `HL7v2IO.Read` to produce a 
`PCollection<HL7v2Message>`
      - `Flatten` the two `PCollection<HL7v2Message>` (one of which will always 
in theory be empty) to form your ultimate output collection
   
   My thoughts of trade-offs are:
   1. Pros: Simpler execution path, simpler to implement (strictly additive) 
given work already done. Cons: User has to know what they're doing and need to 
document / maintain both paths
   1. Pros: Simpler interface for users Cons: more complicated execution path, 
heavier lift.
   
   I'm torn on which would be preferable and would love input on which approach 
others think is cleanest before implementing. (Or suggestions to something I 
haven't thought of!)

----------------------------------------------------------------
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


With regards,
Apache Git Services

Reply via email to