You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by GitBox <gi...@apache.org> on 2020/03/18 01:28:30 UTC

[GitHub] [beam] jaketf opened a new pull request #11151: Hl7v2 io

jaketf opened a new pull request #11151: Hl7v2 io
URL: https://github.com/apache/beam/pull/11151
 
 
   Add HL7v2IO connector for the Google Cloud Healthcare API.
   Initial work for these connectors is being prioritized based to support various 
   customer use cases for HL7v2 -> FHIR ETL pipelines.
   
   Healthcare clients are extremely concerned with not losing data in their pipelines and want dead letter queues for these sources / sinks.
   
   With this in mind the `HL7v2IO.Read` is designed as a mini pipeline that:
     - Starts with an arbitrary `PTransform` to populate a `PCollection` of message IDs. This was chosen to give flexibility between reading the HL7v2 pubsub notifications, reading an entire HL7v2 store (with optional filter) or using some other method (e.g. a hand prepared subset of message IDs uploaded to GCS and read w/ TextIO). In the future we can gain efficiency here when the Healthcare API exposes a batch read method where we can batch up requests rather than making a separate call per element.
     - Then a DoFn that attempts to fetch the actual message contents from the HL7v2 store. Successfully fetched messages will be written to `HL7v2IO.Read.OUT` tag and Failures will be written to a `HLv2IO.Read.DEAD_LETTER` tag. The "expected" failure case would be attempting to fetch a message ID that doesn't exist.
   
   Similarly, the `HL7v2IO.Write` is designed as a mini pipeline that 
     - starts with a `PTransform` that attempts to ingest a `PCollection` of `Messages`
     - then applies the `this.getDeadLetterPTransform()` to the `PCollection` of messages that failed to ingest. This gives the customer the ability to capture dead letters in a system of their chosing (e.g. gcs, bq, pubsub) based on their needs.
   
   - [HL7v2](https://cloud.google.com/healthcare/docs/how-tos/hl7v2-messages):
     - Unbounded Read: Uses PubsubIO to read [notification subscription](https://cloud.google.com/healthcare/docs/how-tos/pubsub#hl7v2_messages)
     - Bounded Read: DoFn to get message IDs using the [Messages.List](https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list) REST API method
   
   Future Work will include a very similar IO transform for FHIR store:
   - [FhirIO](https://cloud.google.com/healthcare/docs/how-tos/fhir-bundles)
     - Write use the [executeBundle](https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/executeBundle) REST API method to execute transactions on the FHIR Store
     - Read with [read](https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/read) / [search](https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/search) / from [notification subscription](https://cloud.google.com/healthcare/docs/how-tos/pubsub#fhir_resources)
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Format the pull request title like `[BEAM-XXX] Fixes bug in ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   Post-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   Lang | SDK | Apex | Dataflow | Flink | Gearpump | Samza | Spark
   --- | --- | --- | --- | --- | --- | --- | ---
   Go | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/)
   Java | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/)
   Python | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/) | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/)
   XLang | --- | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/)
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   --- |Java | Python | Go | Website
   --- | --- | --- | --- | ---
   Non-portable | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/) 
   Portable | --- | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/) | --- | ---
   
   See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs.
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-610705998
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chamikaramj commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614310149
 
 
   You can also consider adding an option to not add the Reshuffle to avoid adding any additional shuffle for anyone who already will have a GBK downstream. See here: https://github.com/apache/beam/blob/master/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java#L823

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r406346420
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   @lukecwik Is there a reason this needs to be `google_clients_version` / 1.30.3?
   I've just tried testing with that version and it breaks most usage of HCAPI.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612261337
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chamikaramj commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r409150054
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,597 @@
+/*
+ * 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.v1beta1.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.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 can be fetched from the HL7v2 store in two ways Message Fetching and Message
+ * Listing.
+ *
+ * <p>Message Fetching
+ *
+ * <p>Message Fetching with {@link HL7v2IO.Read} supports use cases where you have a ${@link
+ * PCollection} of message IDS. This is appropriate for reading the HL7v2 notifications from a
+ * Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually
+ * prepared list of messages that you need to process (e.g. in a text file read with {@link
+ * org.apache.beam.sdk.io.TextIO}) .
+ *
+ * <p>Fetch Message contents from HL7v2 Store based on the {@link PCollection} of message ID strings
+ * {@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},
+ * this can be used to write to the dead letter storage system of your choosing. This error handling
+ * is mainly to catch scenarios where the upstream {@link PCollection} contains IDs that are not
+ * valid or are not reachable due to permissions issues.
+ *
+ * <p>Message Listing Message Listing with {@link HL7v2IO.ListHL7v2Messages} supports batch use
+ * cases where you want to process all the messages in an HL7v2 store or those matching a
+ * filter @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters</a>
+ * This paginates through results of a Messages.List call @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list</a>
+ * and outputs directly to a {@link PCollection} of {@link HL7v2Message}. In these use cases, the
+ * error handling similar to above is unnecessary because we are listing from the source of truth
+ * the pipeline should fail transparently if this transform fails to paginate through all the
+ * results.
+ *
+ * <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} containing the {@link HL7v2Message}
+ * 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 Read Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * PCollection<HL7v2Message> out = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2Messages(
+ *           Collections.singletonList(options.getInputHL7v2Store()), option.getHL7v2Filter()))
+ *    // Go about your happy path transformations.
+ *   .apply("Process HL7v2 Messages", ...);
+ * 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 ListHL7v2Messages} 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} message IDs which failed to be fetched from the HL7v2 store, with
+     *       error message and stacktrace.
+     * </ul>
+     */
+    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 successfulHL7v2MessageGets =
+            Metrics.counter(
+                FetchHL7v2Message.HL7v2MessageGetFn.class, "successful-hl7v2-message-gets");
+        private HealthcareApiClient client;
+
+        /** 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();
+        }
+
+        /**
+         * 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 {
+          long startTime = System.currentTimeMillis();
+
+          com.google.api.services.healthcare.v1beta1.model.Message msg =
+              client.getHL7v2Message(msgId);
+
+          if (msg == null) {
+            throw new IOException(String.format("GET request for %s returned null", msgId));
+          }
+          this.successfulHL7v2MessageGets.inc();
+          return msg;
+        }
+      }
+    }
+  }
+
+  /** List HL7v2 messages. */
+  public static class ListHL7v2Messages extends PTransform<PBegin, PCollection<HL7v2Message>> {
+    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
+     */
+    ListHL7v2Messages(List<String> hl7v2Stores, String filter) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = filter;
+    }
+
+    ListHL7v2Messages(List<String> hl7v2Stores) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = null;
+    }
+
+    @Override
+    public PCollection<HL7v2Message> expand(PBegin input) {
+      return input
+          .apply(Create.of(this.hl7v2Stores))
+          .apply(ParDo.of(new ListHL7v2MessagesFn(this.filter)));
+    }
+  }
+
+  static class ListHL7v2MessagesFn extends DoFn<String, HL7v2Message> {
+
+    private final String filter;
+    private transient HealthcareApiClient client;
+
+    /**
+     * Instantiates a new List HL7v2 fn.
+     *
+     * @param filter the filter
+     */
+    ListHL7v2MessagesFn(String filter) {
+      this.filter = filter;
+    }
+
+    /**
+     * Init client.
+     *
+     * @throws IOException the io exception
+     */
+    @Setup
+    public void initClient() throws IOException {
+      this.client = new HttpHealthcareApiClient();
+    }
+
+    /**
+     * List messages.
+     *
+     * @param context the context
+     * @throws IOException the io exception
+     */
+    @ProcessElement
+    public void listMessages(ProcessContext context) throws IOException {
+      String hl7v2Store = context.element();
+      // Output all elements of all pages.
+      this.client.getHL7v2MessageStream(hl7v2Store, this.filter).forEach(context::output);
 
 Review comment:
   Yeah, this can be a performance bottleneck and this whole operation will be limited to a single machine. Usually sources need an additional level of parallelism due to being high fanout. BTW it might sense to add a Reshuffle at the end here just to allow any subsequent steps to parallelize.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-610050634
 
 
   Next Steps (based on offline feed):
   - [x] Improve API for users:
       - [x] Add static methods for common patterns with `ListHL7v2Messages` 
       - [x] Add `ValueProvider<String>` support to ease use in the DataflowTemplates
           - [x] `ListHL7v2Messages` (hl7v2Store and filter)
           - [x] `Write` (hl7v2Store)
   - [ ] "standardize" integration tests
       - [ ] Refactor ITs to create / destroy HL7v2 Store under a parameterized dataset in `@BeforeClass` `@AfterClass` to avoid issues with parallel tests runs.
       - [ ] Remove hard coding of my HL7v2Store / project in integration tests.
       - [ ] Add Healthcare API Dataset to Beam integration test project

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614337532
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614306261
 
 
   >Yeah, this can be a performance bottleneck and this whole operation will be limited to a single machine. Usually sources need an additional level of parallelism due to being high fanout. BTW it might sense to add a Reshuffle at the end here just to allow any subsequent steps to parallelize.
   
   @chamikaramj that sounds like a good idea. However, I get deprecation working on `org.apache.beam.sdk.transforms.Reshuffle` is there a new blessed way of doing this?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404368939
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,597 @@
+/*
+ * 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.v1beta1.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.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 can be fetched from the HL7v2 store in two ways Message Fetching and Message
+ * Listing.
+ *
+ * <p>Message Fetching
+ *
+ * <p>Message Fetching with {@link HL7v2IO.Read} supports use cases where you have a ${@link
+ * PCollection} of message IDS. This is appropriate for reading the HL7v2 notifications from a
+ * Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually
+ * prepared list of messages that you need to process (e.g. in a text file read with {@link
+ * org.apache.beam.sdk.io.TextIO}) .
+ *
+ * <p>Fetch Message contents from HL7v2 Store based on the {@link PCollection} of message ID strings
+ * {@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},
+ * this can be used to write to the dead letter storage system of your choosing. This error handling
+ * is mainly to catch scenarios where the upstream {@link PCollection} contains IDs that are not
+ * valid or are not reachable due to permissions issues.
+ *
+ * <p>Message Listing Message Listing with {@link HL7v2IO.ListHL7v2Messages} supports batch use
+ * cases where you want to process all the messages in an HL7v2 store or those matching a
+ * filter @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters</a>
+ * This paginates through results of a Messages.List call @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list</a>
+ * and outputs directly to a {@link PCollection} of {@link HL7v2Message}. In these use cases, the
+ * error handling similar to above is unnecessary because we are listing from the source of truth
+ * the pipeline should fail transparently if this transform fails to paginate through all the
+ * results.
+ *
+ * <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} containing the {@link HL7v2Message}
+ * 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 Read Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * PCollection<HL7v2Message> out = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2Messages(
+ *           Collections.singletonList(options.getInputHL7v2Store()), option.getHL7v2Filter()))
+ *    // Go about your happy path transformations.
+ *   .apply("Process HL7v2 Messages", ...);
+ * 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) {
 
 Review comment:
   This is supposed to mirror the [REST API method](https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/ingest) that's used. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lastomato commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lastomato commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-606940440
 
 
   Sorry for the delay. Please see my comments inline.
   
   > Open Questions:
   > 
   > 1. Should we remove adaptive throttling?
   I think it is fine to remove it since the API has quota enabled by default, and retry logic is in place.
   >    
   >    * Seems that we're using retries in the client request initializer and right now a "bad record" will slow down the Read / Write (even though the error has nothing to do with the HL7v2 store being overwhelmed). Originally we wanted to be safe with overwhelming QPS on the HL7v2 store in batch scenarios.
   > 2. Should we add more to the `HealthcareIOError`?
   The second point below would be very helpful, but I am fine with adding it to next PR (if that's easier).
   >    
   >    * Add (processing time) Timestamp?
   >    * Add a convenience DoFn `HealthcareIOErrrorToTableRowFn` to ease writing deadletter queue to BigQuery.
   > 3. Would it be more useful to expose an error rate metric than an error count?
   This functionality is probably already provided by services like stackdriver, we might want to wait until there is a concrete use case.
   
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chamikaramj commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r409150823
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,597 @@
+/*
+ * 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.v1beta1.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.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 can be fetched from the HL7v2 store in two ways Message Fetching and Message
+ * Listing.
+ *
+ * <p>Message Fetching
+ *
+ * <p>Message Fetching with {@link HL7v2IO.Read} supports use cases where you have a ${@link
+ * PCollection} of message IDS. This is appropriate for reading the HL7v2 notifications from a
+ * Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually
+ * prepared list of messages that you need to process (e.g. in a text file read with {@link
+ * org.apache.beam.sdk.io.TextIO}) .
+ *
+ * <p>Fetch Message contents from HL7v2 Store based on the {@link PCollection} of message ID strings
+ * {@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},
+ * this can be used to write to the dead letter storage system of your choosing. This error handling
+ * is mainly to catch scenarios where the upstream {@link PCollection} contains IDs that are not
+ * valid or are not reachable due to permissions issues.
+ *
+ * <p>Message Listing Message Listing with {@link HL7v2IO.ListHL7v2Messages} supports batch use
+ * cases where you want to process all the messages in an HL7v2 store or those matching a
+ * filter @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters</a>
+ * This paginates through results of a Messages.List call @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list</a>
+ * and outputs directly to a {@link PCollection} of {@link HL7v2Message}. In these use cases, the
+ * error handling similar to above is unnecessary because we are listing from the source of truth
+ * the pipeline should fail transparently if this transform fails to paginate through all the
+ * results.
+ *
+ * <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} containing the {@link HL7v2Message}
+ * 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 Read Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * PCollection<HL7v2Message> out = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2Messages(
+ *           Collections.singletonList(options.getInputHL7v2Store()), option.getHL7v2Filter()))
+ *    // Go about your happy path transformations.
+ *   .apply("Process HL7v2 Messages", ...);
+ * 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 ListHL7v2Messages} 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} message IDs which failed to be fetched from the HL7v2 store, with
+     *       error message and stacktrace.
+     * </ul>
+     */
+    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 successfulHL7v2MessageGets =
+            Metrics.counter(
+                FetchHL7v2Message.HL7v2MessageGetFn.class, "successful-hl7v2-message-gets");
+        private HealthcareApiClient client;
+
+        /** 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();
+        }
+
+        /**
+         * 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 {
+          long startTime = System.currentTimeMillis();
+
+          com.google.api.services.healthcare.v1beta1.model.Message msg =
+              client.getHL7v2Message(msgId);
+
+          if (msg == null) {
+            throw new IOException(String.format("GET request for %s returned null", msgId));
+          }
+          this.successfulHL7v2MessageGets.inc();
+          return msg;
+        }
+      }
+    }
+  }
+
+  /** List HL7v2 messages. */
+  public static class ListHL7v2Messages extends PTransform<PBegin, PCollection<HL7v2Message>> {
+    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
+     */
+    ListHL7v2Messages(List<String> hl7v2Stores, String filter) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = filter;
+    }
+
+    ListHL7v2Messages(List<String> hl7v2Stores) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = null;
+    }
+
+    @Override
+    public PCollection<HL7v2Message> expand(PBegin input) {
+      return input
+          .apply(Create.of(this.hl7v2Stores))
+          .apply(ParDo.of(new ListHL7v2MessagesFn(this.filter)));
+    }
+  }
+
+  static class ListHL7v2MessagesFn extends DoFn<String, HL7v2Message> {
+
+    private final String filter;
+    private transient HealthcareApiClient client;
+
+    /**
+     * Instantiates a new List HL7v2 fn.
+     *
+     * @param filter the filter
+     */
+    ListHL7v2MessagesFn(String filter) {
+      this.filter = filter;
+    }
+
+    /**
+     * Init client.
+     *
+     * @throws IOException the io exception
+     */
+    @Setup
+    public void initClient() throws IOException {
+      this.client = new HttpHealthcareApiClient();
+    }
+
+    /**
+     * List messages.
+     *
+     * @param context the context
+     * @throws IOException the io exception
+     */
+    @ProcessElement
+    public void listMessages(ProcessContext context) throws IOException {
+      String hl7v2Store = context.element();
+      // Output all elements of all pages.
+      this.client.getHL7v2MessageStream(hl7v2Store, this.filter).forEach(context::output);
+    }
+  }
+
+  /** The type Write. */
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<HL7v2Message>, Write.Result> {
+
+    /** The tag for the successful writes to HL7v2 store`. */
+    public static final TupleTag<HealthcareIOError<HL7v2Message>> SUCCESS =
+        new TupleTag<HealthcareIOError<HL7v2Message>>() {};
+    /** The tag for the failed writes to HL7v2 store`. */
+    public static final TupleTag<HealthcareIOError<HL7v2Message>> FAILED =
+        new TupleTag<HealthcareIOError<HL7v2Message>>() {};
+
+    /**
+     * Gets HL7v2 store.
+     *
+     * @return the HL7v2 store
+     */
+    abstract String getHL7v2Store();
+
+    /**
+     * Gets write method.
+     *
+     * @return the write method
+     */
+    abstract WriteMethod getWriteMethod();
+
+    @Override
+    public Result expand(PCollection<HL7v2Message> messages) {
+      return messages.apply(new WriteHL7v2(this.getHL7v2Store(), this.getWriteMethod()));
+    }
+
+    /** The enum Write method. */
+    public enum WriteMethod {
+      /**
+       * Ingest write method. @see <a
+       * href=https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/ingest></a>
+       */
+      INGEST,
+      /**
+       * Batch import write method. This is not yet supported by the HL7v2 API, but can be used to
+       * improve throughput once available.
+       */
+      BATCH_IMPORT
+    }
+
+    /** The type Builder. */
+    @AutoValue.Builder
+    abstract static class Builder {
+
+      /**
+       * Sets HL7v2 store.
+       *
+       * @param hl7v2Store the HL7v2 store
+       * @return the HL7v2 store
+       */
+      abstract Builder setHL7v2Store(String hl7v2Store);
+
+      /**
+       * Sets write method.
+       *
+       * @param writeMethod the write method
+       * @return the write method
+       */
+      abstract Builder setWriteMethod(WriteMethod writeMethod);
+
+      /**
+       * Build write.
+       *
+       * @return the write
+       */
+      abstract Write build();
+    }
+
+    public static class Result implements POutput {
+      private final Pipeline pipeline;
+      private final PCollection<HealthcareIOError<HL7v2Message>> failedInsertsWithErr;
+
+      /** Creates a {@link HL7v2IO.Write.Result} in the given {@link Pipeline}. */
+      static Result in(
+          Pipeline pipeline, PCollection<HealthcareIOError<HL7v2Message>> failedInserts) {
+        return new Result(pipeline, failedInserts);
+      }
+
+      public PCollection<HealthcareIOError<HL7v2Message>> getFailedInsertsWithErr() {
+        return this.failedInsertsWithErr;
+      }
+
+      @Override
+      public Pipeline getPipeline() {
+        return this.pipeline;
+      }
+
+      @Override
+      public Map<TupleTag<?>, PValue> expand() {
+        failedInsertsWithErr.setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder()));
+        return ImmutableMap.of(FAILED, failedInsertsWithErr);
+      }
+
+      @Override
+      public void finishSpecifyingOutput(
+          String transformName, PInput input, PTransform<?, ?> transform) {}
+
+      private Result(
+          Pipeline pipeline, PCollection<HealthcareIOError<HL7v2Message>> failedInsertsWithErr) {
+        this.pipeline = pipeline;
+        this.failedInsertsWithErr = failedInsertsWithErr;
+      }
+    }
+  }
+
+  /** The type Write hl 7 v 2. */
+  static class WriteHL7v2 extends PTransform<PCollection<HL7v2Message>, Write.Result> {
+    private final String hl7v2Store;
+    private final Write.WriteMethod writeMethod;
+
+    /**
+     * Instantiates a new Write hl 7 v 2.
+     *
+     * @param hl7v2Store the hl 7 v 2 store
+     * @param writeMethod the write method
+     */
+    WriteHL7v2(String hl7v2Store, Write.WriteMethod writeMethod) {
+      this.hl7v2Store = hl7v2Store;
+      this.writeMethod = writeMethod;
+    }
+
+    @Override
+    public Write.Result expand(PCollection<HL7v2Message> input) {
+      PCollection<HealthcareIOError<HL7v2Message>> failedInserts =
+          input
+              .apply(ParDo.of(new WriteHL7v2Fn(hl7v2Store, writeMethod)))
+              .setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder()));
+      return Write.Result.in(input.getPipeline(), failedInserts);
+    }
+
+    /** The type Write hl 7 v 2 fn. */
+    static class WriteHL7v2Fn extends DoFn<HL7v2Message, HealthcareIOError<HL7v2Message>> {
+      // TODO when the healthcare API releases a bulk import method this should use that to improve
+      // throughput.
+
+      private Counter failedMessageWrites =
+          Metrics.counter(WriteHL7v2Fn.class, "failed-hl7v2-message-writes");
+      private final String hl7v2Store;
+      private final Counter successfulHL7v2MessageWrites =
+          Metrics.counter(WriteHL7v2.class, "successful-hl7v2-message-writes");
+      private final Write.WriteMethod writeMethod;
+
+      private static final Logger LOG = LoggerFactory.getLogger(WriteHL7v2.WriteHL7v2Fn.class);
+      private transient HealthcareApiClient client;
+
+      /**
+       * Instantiates a new Write HL7v2 fn.
+       *
+       * @param hl7v2Store the HL7v2 store
+       * @param writeMethod the write method
+       */
+      WriteHL7v2Fn(String hl7v2Store, Write.WriteMethod writeMethod) {
+        this.hl7v2Store = hl7v2Store;
+        this.writeMethod = writeMethod;
+      }
+
+      /**
+       * Init client.
+       *
+       * @throws IOException the io exception
+       */
+      @Setup
+      public void initClient() throws IOException {
+        this.client = new HttpHealthcareApiClient();
+      }
+
+      /**
+       * Write messages.
+       *
+       * @param context the context
+       */
+      @ProcessElement
+      public void writeMessages(ProcessContext context) {
+        HL7v2Message msg = context.element();
+        long startTime = System.currentTimeMillis();
+        Sleeper sleeper = Sleeper.DEFAULT;
+        switch (writeMethod) {
+          case BATCH_IMPORT:
+            throw new UnsupportedOperationException("The Batch import API is not available yet");
+          case INGEST:
+          default:
+            try {
+              client.ingestHL7v2Message(hl7v2Store, msg.toModel());
 
 Review comment:
   Please add a TODO for this. I think this is a critical performance bottleneck.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614237933
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404366818
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOErrorCoder.java
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * 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 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;
+
+public class HealthcareIOErrorCoder<T> extends CustomCoder<HealthcareIOError<T>> {
+  private final Coder<T> originalCoder;
+  private static final NullableCoder<String> STRING_CODER = NullableCoder.of(StringUtf8Coder.of());
+
+  HealthcareIOErrorCoder(Coder<T> originalCoder) {
+    this.originalCoder = NullableCoder.of(originalCoder);
+  }
+
+  @Override
+  public void encode(HealthcareIOError<T> value, OutputStream outStream)
+      throws CoderException, IOException {
+
+    originalCoder.encode(value.getDataResource(), outStream);
+
 
 Review comment:
   Good Catch.
   We should serialize / deserialize observed time, no reason to introduce that drift even if it is small.
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-605212812
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-603980265
 
 
   Open Questions:
   1. Should we remove adaptive throttling?
       - Seems that we're using retries in the client request initializer and right now a "bad record" will slow down the Read / Write (even though the error has nothing to do with the HL7v2 store being overwhelmed). Originally we wanted to be safe with overwhelming QPS on the HL7v2 store in batch scenarios.
   1. Should we add more to the `HealthcareIOError`?
       - Add (processing time) Timestamp?
       - Add a convenience DoFn `HealthcareIOErrrorToTableRowFn` to ease writing deadletter queue to BigQuery.
   1. Would it me more useful to expose an error rate metric than an error count?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r395946114
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,620 @@
+/*
+ * 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.IngestMessageResponse;
+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.Collections;
+import java.util.List;
+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.PDone;
+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.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 HL7v2 Messages are fetched from the HL7v2 store based on the {@link PCollection} of of
+ * message IDs {@link String}s produced by the {@link AutoValue_HL7v2IO_Read#getMessageIDTransform}
+ * as {@link PCollectionTuple}*** containing an {@link HL7v2IO.Read#OUT} tag for successfully
+ * fetched messages and a {@link HL7v2IO.Read#DEAD_LETTER} tag for message IDs that could not be
+ * fetched.
+ *
+ * <p>HL7v2 stores can be read in several ways: - Unbounded: based on the Pub/Sub Notification
+ * Channel {@link HL7v2IO#readNotificationSubscription(String)} - Bounded: based on reading an
+ * entire HL7v2 store (or stores) {@link HL7v2IO#readHL7v2Store(String)} - Bounded: based on reading
+ * an HL7v2 store with a filter
+ *
+ * <p>Note, due to the flexibility of this Read transform, this must output a dead letter queue.
+ * This handles the scenario where the the PTransform that populates a PCollection of message IDs
+ * contains message IDs that do not exist in the HL7v2 stores.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline pipeline = Pipeline.create(options)
+ *
+ *
+ * PCollectionTuple messages = pipeline.apply(
+ *     new HLv2IO.readNotifications(options.getNotificationSubscription())
+ *
+ * // Write errors to your favorite dead letter  queue (e.g. Pub/Sub, GCS, BigQuery)
+ * messages.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.
+ * PCollection<Message> out = fetchedMessages.apply("ProcessFetchedMessages", ...);
+ *
+ * // Write using the Message.Ingest method of the HL7v2 REST API.
+ * out.apply(HL7v2IO.ingestMessages(options.getOutputHL7v2Store()));
+ *
+ * pipeline.run();
+ *
+ * }***
+ * </pre>
+ */
+public class HL7v2IO {
+  // TODO add metrics for failed records.
+
+  private static Read.Builder read(PTransform<PBegin, PCollection<String>> messageIDTransform) {
 
 Review comment:
   refactored to implement both  `HL7v2IO.[Read,Write].Result`

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613067296
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614243069
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614241615
 
 
   retest this please
   
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-603441209
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404353084
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,597 @@
+/*
+ * 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.v1beta1.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.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 can be fetched from the HL7v2 store in two ways Message Fetching and Message
+ * Listing.
+ *
+ * <p>Message Fetching
+ *
+ * <p>Message Fetching with {@link HL7v2IO.Read} supports use cases where you have a ${@link
+ * PCollection} of message IDS. This is appropriate for reading the HL7v2 notifications from a
+ * Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually
+ * prepared list of messages that you need to process (e.g. in a text file read with {@link
+ * org.apache.beam.sdk.io.TextIO}) .
+ *
+ * <p>Fetch Message contents from HL7v2 Store based on the {@link PCollection} of message ID strings
+ * {@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},
+ * this can be used to write to the dead letter storage system of your choosing. This error handling
+ * is mainly to catch scenarios where the upstream {@link PCollection} contains IDs that are not
+ * valid or are not reachable due to permissions issues.
+ *
+ * <p>Message Listing Message Listing with {@link HL7v2IO.ListHL7v2Messages} supports batch use
+ * cases where you want to process all the messages in an HL7v2 store or those matching a
+ * filter @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters</a>
+ * This paginates through results of a Messages.List call @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list</a>
+ * and outputs directly to a {@link PCollection} of {@link HL7v2Message}. In these use cases, the
+ * error handling similar to above is unnecessary because we are listing from the source of truth
+ * the pipeline should fail transparently if this transform fails to paginate through all the
+ * results.
+ *
+ * <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} containing the {@link HL7v2Message}
+ * 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 Read Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * PCollection<HL7v2Message> out = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2Messages(
+ *           Collections.singletonList(options.getInputHL7v2Store()), option.getHL7v2Filter()))
+ *    // Go about your happy path transformations.
+ *   .apply("Process HL7v2 Messages", ...);
+ * 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 ListHL7v2Messages} 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
 
 Review comment:
   I guess it's not a DoFn?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
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 new list API to save the extra retrieval.
   
   IMO this DoFn should remain as is to support backwards compatibility with the beta 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 old api requiring "get" after list: `HL7v2IO.ListHL7v2MessageIDs` -> `HL7v2IO.Read`(current path).
      - Add a new PTransform for new 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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612500826
 
 
   Run Java PostCommit
   
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-603980265
 
 
   Open Questions:
   1. Should we remove adaptive throttling?
       - Seems that we're using retries in the client request initializer and right now a "bad record" will slow down the Read / Write (even though the error has nothing to do with the HL7v2 store being overwhelmed). Originally we wanted to be safe with overwhelming QPS on the HL7v2 store in batch scenarios.
   1. Should we add more to the `HealthcareIOError`?
       - Add (processing time) Timestamp?
       - Add a convenience DoFn `HealthcareIOErrrorToTableRowFn` to ease writing deadletter queue to BigQuery.
   1. Would it be more useful to expose an error rate metric than an error count?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-604655037
 
 
   Ok an updates here from an internal thread w/ API team.
   
   1. Message.List returning message contents is available in beta API with the view parameter.
   1. Schematized Data should be in next beta release roughly in ~2 weeks.
   1. right now the sink is outputting schematized data json wrapped in "{data=<actual_valid_json>}" 
   
   In light of these I will do the following refactors: 
   1. how we batch read from to always avoid the double get. This will make it a completely parallel code path than the real-time path but I think that's ok.
   1. refactor to use beta client library
   1. I'll strip out that `{data=}` wrapper to make this easier for users.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612242492
 
 
   Thanks for the pointer!
   I will give that a try.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-603996758
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-604655037
 
 
   Ok an updates here from an internal thread w/ API team.
   
   1. [Message.List returning message contents is available in beta API with the view parameter.
   1. Schematized Data should be in next beta release roughly in ~2 weeks.
   1. right now the sink is outputting schematized data json wrapped in "{data=<actual_valid_json>}" 
   
   In light of these I will do the following refactors: 
   1. [x] how we batch read from to always avoid the double get. This will make it a completely parallel code path than the real-time path but I think that's ok.
   1. [ ] refactor to use beta client library (once it includes schematizedData)
       - FYI pre-work for v1beta1 migration in this branch [migration/v1beta1](https://github.com/jaketf/beam/tree/migration/v1beta1). Will merge back to this branch once we have schematizedData in beta client library.
   1. [x] I'll strip out that `{data=}` wrapper to make this easier for users.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] suztomo commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
suztomo commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r409057698
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java
 ##########
 @@ -0,0 +1,459 @@
+/*
+ * 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.client.http.HttpHeaders;
+import com.google.api.client.http.HttpRequest;
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.client.http.javanet.NetHttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.healthcare.v1beta1.CloudHealthcare;
+import com.google.api.services.healthcare.v1beta1.CloudHealthcare.Projects.Locations.Datasets.Hl7V2Stores.Messages;
+import com.google.api.services.healthcare.v1beta1.CloudHealthcareScopes;
+import com.google.api.services.healthcare.v1beta1.model.CreateMessageRequest;
+import com.google.api.services.healthcare.v1beta1.model.Empty;
+import com.google.api.services.healthcare.v1beta1.model.Hl7V2Store;
+import com.google.api.services.healthcare.v1beta1.model.HttpBody;
+import com.google.api.services.healthcare.v1beta1.model.IngestMessageRequest;
+import com.google.api.services.healthcare.v1beta1.model.IngestMessageResponse;
+import com.google.api.services.healthcare.v1beta1.model.ListMessagesResponse;
+import com.google.api.services.healthcare.v1beta1.model.Message;
+import com.google.api.services.healthcare.v1beta1.model.SearchResourcesRequest;
+import com.google.auth.oauth2.GoogleCredentials;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URI;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Spliterator;
+import java.util.Spliterators;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.gcp.util.RetryHttpRequestInitializer;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
+
+/**
+ * A client that talks to the Cloud Healthcare API through HTTP requests. This client is created
+ * mainly to encapsulate the unserializable dependencies, since most generated classes are not
+ * serializable in the HTTP client.
+ */
+public class HttpHealthcareApiClient<T> implements HealthcareApiClient, Serializable {
+
+  private transient CloudHealthcare client;
+
+  /**
+   * Instantiates a new Http healthcare api client.
+   *
+   * @throws IOException the io exception
+   */
+  public HttpHealthcareApiClient() throws IOException {
+    initClient();
+  }
+
+  /**
+   * Instantiates a new Http healthcare api client.
+   *
+   * @param client the client
+   * @throws IOException the io exception
+   */
+  public HttpHealthcareApiClient(CloudHealthcare client) throws IOException {
+    this.client = client;
+    initClient();
+  }
+
+  @VisibleForTesting
+  static <T, X extends Collection<T>> Stream<T> flattenIteratorCollectionsToStream(
+      Iterator<X> iterator) {
+    Spliterator<Collection<T>> spliterator = Spliterators.spliteratorUnknownSize(iterator, 0);
+    return StreamSupport.stream(spliterator, false).flatMap(Collection::stream);
+  }
+
+  public JsonFactory getJsonFactory() {
+    return this.client.getJsonFactory();
+  }
+
+  @Override
+  public Hl7V2Store createHL7v2Store(String dataset, String name) throws IOException {
+    Hl7V2Store store = new Hl7V2Store();
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .create(dataset, store)
+        .setHl7V2StoreId(name)
+        .execute();
+  }
+
+  @Override
+  public Empty deleteHL7v2Store(String name) throws IOException {
+    return client.projects().locations().datasets().hl7V2Stores().delete(name).execute();
+  }
+
+  @Override
+  public ListMessagesResponse makeHL7v2ListRequest(
+      String hl7v2Store, @Nullable String filter, @Nullable String pageToken) throws IOException {
+
+    Messages.List baseRequest =
+        client
+            .projects()
+            .locations()
+            .datasets()
+            .hl7V2Stores()
+            .messages()
+            .list(hl7v2Store)
+            .set("view", "full")
+            .setPageToken(pageToken);
+
+    if (Strings.isNullOrEmpty(filter)) {
+      return baseRequest.execute();
+    } else {
+      return baseRequest.setFilter(filter).execute();
+    }
+  }
+
+  /**
+   * Gets message id page iterator.
+   *
+   * @param hl7v2Store the HL7v2 store
+   * @return the message id page iterator
+   * @throws IOException the io exception
+   */
+  @Override
+  public Stream<HL7v2Message> getHL7v2MessageStream(String hl7v2Store) throws IOException {
+    return getHL7v2MessageStream(hl7v2Store, null);
+  }
+
+  /**
+   * Get a {@link Stream} of message IDs from flattening the pages of a new {@link
+   * HL7v2MessagePages}.
+   *
+   * @param hl7v2Store the HL7v2 store
+   * @param filter the filter
+   * @return the message id Stream
+   * @throws IOException the io exception
+   */
+  @Override
+  public Stream<HL7v2Message> getHL7v2MessageStream(String hl7v2Store, @Nullable String filter)
+      throws IOException {
+    Iterator<List<HL7v2Message>> iterator =
+        new HL7v2MessagePages(this, hl7v2Store, filter).iterator();
+    return flattenIteratorCollectionsToStream(iterator);
+  }
+
+  /**
+   * Gets HL7v2 message.
+   *
+   * @param msgName the msg name
+   * @return the message
+   * @throws IOException the io exception
+   * @throws ParseException the parse exception
+   */
+  @Override
+  public Message getHL7v2Message(String msgName) throws IOException {
+    Message msg =
+        client.projects().locations().datasets().hl7V2Stores().messages().get(msgName).execute();
+    if (msg == null) {
+      throw new IOException(String.format("Couldn't find message: %s.", msgName));
+    }
+    return msg;
+  }
+
+  @Override
+  public Empty deleteHL7v2Message(String msgName) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .delete(msgName)
+        .execute();
+  }
+
+  /**
+   * Gets HL7v2 store.
+   *
+   * @param storeName the store name
+   * @return the HL7v2 store
+   * @throws IOException the io exception
+   */
+  @Override
+  public Hl7V2Store getHL7v2Store(String storeName) throws IOException {
+    return client.projects().locations().datasets().hl7V2Stores().get(storeName).execute();
+  }
+
+  @Override
+  public IngestMessageResponse ingestHL7v2Message(String hl7v2Store, Message msg)
+      throws IOException {
+    IngestMessageRequest ingestMessageRequest = new IngestMessageRequest();
+    ingestMessageRequest.setMessage(msg);
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .ingest(hl7v2Store, ingestMessageRequest)
+        .execute();
+  }
+
+  @Override
+  public HttpBody fhirSearch(String fhirStore, SearchResourcesRequest query) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .search(fhirStore, query)
+        .execute();
+  }
+
+  @Override
+  public Message createHL7v2Message(String hl7v2Store, Message msg) throws IOException {
+    CreateMessageRequest createMessageRequest = new CreateMessageRequest();
+    createMessageRequest.setMessage(msg);
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .create(hl7v2Store, createMessageRequest)
+        .execute();
+  }
+
+  @Override
+  public HttpBody createFhirResource(String fhirStore, String type, HttpBody body)
+      throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .create(fhirStore, type, body)
+        .execute();
+  }
+
+  @Override
+  public HttpBody executeFhirBundle(String fhirStore, HttpBody bundle) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .executeBundle(fhirStore, bundle)
+        .execute();
+  }
+
+  @Override
+  public HttpBody listFHIRResourceForPatient(String fhirStore, String patient) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .patientEverything(patient)
+        .execute();
+  }
+
+  @Override
+  public HttpBody readFHIRResource(String fhirStore, String resource) throws IOException {
+    return client.projects().locations().datasets().fhirStores().fhir().read(resource).execute();
+  }
+
+  private static class AuthenticatedRetryInitializer extends RetryHttpRequestInitializer {
+    GoogleCredentials credentials;
+
+    public AuthenticatedRetryInitializer(GoogleCredentials credentials) {
+      super();
+      this.credentials = credentials;
+    }
+
+    @Override
+    public void initialize(HttpRequest request) throws IOException {
+      super.initialize(request);
+      if (!credentials.hasRequestMetadata()) {
+        return;
+      }
+      HttpHeaders requestHeaders = request.getHeaders();
+      requestHeaders.setUserAgent("apache-beam-hl7v2-io");
+      URI uri = null;
+      if (request.getUrl() != null) {
+        uri = request.getUrl().toURI();
+      }
+      Map<String, List<String>> credentialHeaders = credentials.getRequestMetadata(uri);
+      if (credentialHeaders == null) {
+        return;
+      }
+      for (Map.Entry<String, List<String>> entry : credentialHeaders.entrySet()) {
+        String headerName = entry.getKey();
+        List<String> requestValues = new ArrayList<>(entry.getValue());
+        requestHeaders.put(headerName, requestValues);
+      }
+    }
+  }
+
+  private void initClient() throws IOException {
+    // Create a HttpRequestInitializer, which will provide a baseline configuration to all requests.
+    // HttpRequestInitializer requestInitializer = new RetryHttpRequestInitializer();
+    // GoogleCredentials credentials = GoogleCredentials.getApplicationDefault();
+    HttpRequestInitializer requestInitializer =
+        new AuthenticatedRetryInitializer(
+            GoogleCredentials.getApplicationDefault()
+                .createScoped(CloudHealthcareScopes.CLOUD_PLATFORM));
+
+    client =
+        new CloudHealthcare.Builder(new NetHttpTransport(), new GsonFactory(), requestInitializer)
 
 Review comment:
   I see CloudHealthcare.Builder accepts JacksonFactory.
   https://github.com/googleapis/google-api-java-client-services/blob/master/clients/google-api-services-healthcare/v1beta1/1.30.1/com/google/api/services/healthcare/v1beta1/CloudHealthcare.java#L16349

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613150119
 
 
   this looks fine to me as long as the dependency changes look fine to @lukecwik 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-606147221
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612237829
 
 
   @lukecwik that's what I thought to but when I do that I get the output in [this comment](https://github.com/apache/beam/pull/11151#discussion_r406551322)

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612242492
 
 
   Thanks for the very concrete actionable pointer!
   I will give that a try.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r406330770
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   It'd make this PR cleaner to use this most recent version because [this commit adds `SchematizedData`](https://github.com/googleapis/google-api-java-client-services/commit/48db535eb30fea17c615f9951d7afad6f3ea72b4#diff-89048b80fe6bcee9b0aade84024aea66).
   
   which I am currently around [here](https://github.com/apache/beam/pull/11151/files#diff-2173a334dc15f8b7dc3257efeec64504R27).

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-604655037
 
 
   Ok an updates here from an internal thread w/ API team.
   
   1. [Message.List returning message contents is available in beta API with the view parameter.
   1. Schematized Data should be in next beta release roughly in ~2 weeks.
   1. right now the sink is outputting schematized data json wrapped in "{data=<actual_valid_json>}" 
   
   In light of these I will do the following refactors: 
   1. [x] how we batch read from to always avoid the double get. This will make it a completely parallel code path than the real-time path but I think that's ok.
   1. [x] refactor to use beta client library (once it includes schematizedData)
       - FYI pre-work for v1beta1 migration in this branch [migration/v1beta1](https://github.com/jaketf/beam/tree/migration/v1beta1). Will merge back to this branch once we have schematizedData in beta client library.
   1. [x] I'll strip out that `{data=}` wrapper to make this easier for users.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chamikaramj commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614309223
 
 
   You can ignore the deprecation warning for Reshuffle.viaRandomKey(). The deprecation was just because the behavior across runners for Reshuffle is not well defined. Transform is not going away. Many runners add a fusion break after GBK. So this will allow subsequent steps to parallelize better.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-604060180
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] brianlucier commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
brianlucier commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r398180848
 
 

 ##########
 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:
   It might be worth supporting a the extra options on the new list API (view, and orderBy).  In particular, the view can return the full message payload right in the list, thereby saving a future retrieval step.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r409072416
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -469,6 +470,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_cloud_datastore_v1_proto_client      : "com.google.cloud.datastore:datastore-v1-proto-client:1.6.3",
         google_cloud_spanner                        : "com.google.cloud:google-cloud-spanner:$google_cloud_spanner_version",
         google_http_client                          : "com.google.http-client:google-http-client:$google_http_clients_version",
+        google_http_client_gson                     : "com.google.http-client:google-http-client-gson:$google_http_clients_version",
 
 Review comment:
   good question.
   I used Gson to be consistent w/ how we do things in the healthcare example docs and a CDAP plugin that was developed for the same purpose.
   It does not seem strictly necessary.
   I will prepare a commit to remove gson dep and migrate to jackson.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-610050634
 
 
   Next Steps (based on offline feed):
   - [x] Improve API for users:
       - [x] Add static methods for common patterns with `ListHL7v2Messages` 
       - [x] Add `ValueProvider<String>` support to ease use in the DataflowTemplates
           - [x] `ListHL7v2Messages` (hl7v2Store and filter)
           - [x] `Write` (hl7v2Store)
   - [ ] "standardize" integration tests
       - [x] Refactor ITs to create / destroy HL7v2 Store under a parameterized dataset in `@BeforeClass` `@AfterClass` to avoid issues with parallel tests runs.
       - [ ] Remove hard coding of my HL7v2Store / project in integration tests.
       - [ ] Add Healthcare API Dataset to Beam integration test project

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem merged pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem merged pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151
 
 
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-600827169
 
 
   since your change is a Java change, you don't need to worry about the Python errors for now. They seem to come from upstream.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r406327077
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   Good question!
   Please let me know if I'm misunderstanding but I believe that is [set to 1.30.3](https://github.com/apache/beam/blob/master/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy#L379)
   And looking at this [repo for healthcare client](https://github.com/googleapis/google-api-java-client-services/tree/master/clients/google-api-services-healthcare/v1beta1) there doesn't seem to be anything past 1.30.1 (which was regenerated yesterday).
   
   Also FYI you're commenting on an outdated commit we are now using beta client lib.
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-611213022
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404372529
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,636 @@
+/*
+ * 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
 
 Review comment:
   @pabloem please see [descripion](https://github.com/apache/beam/pull/11151/files/ba9d0239dea565b76e07546596d4e52ccf69bd4d#diff-9cd595f078378218ccc01ce7e19ca766R78-R87) and [example](https://github.com/apache/beam/pull/11151/files/ba9d0239dea565b76e07546596d4e52ccf69bd4d#diff-9cd595f078378218ccc01ce7e19ca766R131-R137) added to this javadoc string.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612242130
 
 
   > @lukecwik that's what I thought to but when I do that I get the output in [this comment](https://github.com/apache/beam/pull/11151#discussion_r406551322)
   
   A lot of google libraries have three parts to the version string (API major version, API minor version, shared lib version)
   
   You have to find and upgrade the client libraries to versions that contain the 1.30.9 shared lib version like:
   google-api-services-cloudresourcemanager:v1-rev20191206-1.30.4 becomes google-api-services-cloudresourcemanager:v1-rev20200311-1.30.9
   
   Best way to find matching versions is to look at the index of the maven repo for each artifact (e.g.):
   https://repo1.maven.org/maven2/com/google/apis/google-api-services-cloudresourcemanager/
   
   Find ones that have the same API major version and any newer API minor version with the shard lib version being 1.30.9.
   
   Major version changes are breaking API changes, minor version increases are meant to be safe to upgrade and shared lib version just needs to be the same across all deps.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-603441156
 
 
   let me know if this is ready for review or not

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613058477
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613067175
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404374442
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,597 @@
+/*
+ * 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.v1beta1.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.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 can be fetched from the HL7v2 store in two ways Message Fetching and Message
+ * Listing.
+ *
+ * <p>Message Fetching
+ *
+ * <p>Message Fetching with {@link HL7v2IO.Read} supports use cases where you have a ${@link
+ * PCollection} of message IDS. This is appropriate for reading the HL7v2 notifications from a
+ * Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually
+ * prepared list of messages that you need to process (e.g. in a text file read with {@link
+ * org.apache.beam.sdk.io.TextIO}) .
+ *
+ * <p>Fetch Message contents from HL7v2 Store based on the {@link PCollection} of message ID strings
+ * {@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},
+ * this can be used to write to the dead letter storage system of your choosing. This error handling
+ * is mainly to catch scenarios where the upstream {@link PCollection} contains IDs that are not
+ * valid or are not reachable due to permissions issues.
+ *
+ * <p>Message Listing Message Listing with {@link HL7v2IO.ListHL7v2Messages} supports batch use
+ * cases where you want to process all the messages in an HL7v2 store or those matching a
+ * filter @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters</a>
+ * This paginates through results of a Messages.List call @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list</a>
+ * and outputs directly to a {@link PCollection} of {@link HL7v2Message}. In these use cases, the
+ * error handling similar to above is unnecessary because we are listing from the source of truth
+ * the pipeline should fail transparently if this transform fails to paginate through all the
+ * results.
+ *
+ * <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} containing the {@link HL7v2Message}
+ * 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 Read Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * PCollection<HL7v2Message> out = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2Messages(
+ *           Collections.singletonList(options.getInputHL7v2Store()), option.getHL7v2Filter()))
+ *    // Go about your happy path transformations.
+ *   .apply("Process HL7v2 Messages", ...);
+ * 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 ListHL7v2Messages} 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} message IDs which failed to be fetched from the HL7v2 store, with
+     *       error message and stacktrace.
+     * </ul>
+     */
+    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 successfulHL7v2MessageGets =
+            Metrics.counter(
+                FetchHL7v2Message.HL7v2MessageGetFn.class, "successful-hl7v2-message-gets");
+        private HealthcareApiClient client;
+
+        /** 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();
+        }
+
+        /**
+         * 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 {
+          long startTime = System.currentTimeMillis();
+
+          com.google.api.services.healthcare.v1beta1.model.Message msg =
+              client.getHL7v2Message(msgId);
+
+          if (msg == null) {
+            throw new IOException(String.format("GET request for %s returned null", msgId));
+          }
+          this.successfulHL7v2MessageGets.inc();
+          return msg;
+        }
+      }
+    }
+  }
+
+  /** List HL7v2 messages. */
+  public static class ListHL7v2Messages extends PTransform<PBegin, PCollection<HL7v2Message>> {
+    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
+     */
+    ListHL7v2Messages(List<String> hl7v2Stores, String filter) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = filter;
+    }
+
+    ListHL7v2Messages(List<String> hl7v2Stores) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = null;
+    }
+
+    @Override
+    public PCollection<HL7v2Message> expand(PBegin input) {
+      return input
+          .apply(Create.of(this.hl7v2Stores))
+          .apply(ParDo.of(new ListHL7v2MessagesFn(this.filter)));
+    }
+  }
+
+  static class ListHL7v2MessagesFn extends DoFn<String, HL7v2Message> {
+
+    private final String filter;
+    private transient HealthcareApiClient client;
+
+    /**
+     * Instantiates a new List HL7v2 fn.
+     *
+     * @param filter the filter
+     */
+    ListHL7v2MessagesFn(String filter) {
+      this.filter = filter;
+    }
+
+    /**
+     * Init client.
+     *
+     * @throws IOException the io exception
+     */
+    @Setup
+    public void initClient() throws IOException {
+      this.client = new HttpHealthcareApiClient();
+    }
+
+    /**
+     * List messages.
+     *
+     * @param context the context
+     * @throws IOException the io exception
+     */
+    @ProcessElement
+    public void listMessages(ProcessContext context) throws IOException {
+      String hl7v2Store = context.element();
+      // Output all elements of all pages.
+      this.client.getHL7v2MessageStream(hl7v2Store, this.filter).forEach(context::output);
 
 Review comment:
   The message listing implementation in `HttpHealthcareApiClient.HL7v2MessagePages` iterator will be a bottleneck here as it has to paginate through the list results in a single thread. The API does not support reading from an offset.
   
   The choice for stream here should eagerly output results as each page is retrieved from the API rather than blocking until all pages have been read.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404334588
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOErrorCoder.java
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * 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 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;
+
+public class HealthcareIOErrorCoder<T> extends CustomCoder<HealthcareIOError<T>> {
+  private final Coder<T> originalCoder;
+  private static final NullableCoder<String> STRING_CODER = NullableCoder.of(StringUtf8Coder.of());
+
+  HealthcareIOErrorCoder(Coder<T> originalCoder) {
+    this.originalCoder = NullableCoder.of(originalCoder);
+  }
+
+  @Override
+  public void encode(HealthcareIOError<T> value, OutputStream outStream)
+      throws CoderException, IOException {
+
+    originalCoder.encode(value.getDataResource(), outStream);
+
 
 Review comment:
   I see the observedTime is not serialized/deserialized. This means that the observedTime for a HealthcareIOError event will keep changing as the event is serialized / deserialized. Though I guess it will be roughly accurate when it's written out. WDYT?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r409072416
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -469,6 +470,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_cloud_datastore_v1_proto_client      : "com.google.cloud.datastore:datastore-v1-proto-client:1.6.3",
         google_cloud_spanner                        : "com.google.cloud:google-cloud-spanner:$google_cloud_spanner_version",
         google_http_client                          : "com.google.http-client:google-http-client:$google_http_clients_version",
+        google_http_client_gson                     : "com.google.http-client:google-http-client-gson:$google_http_clients_version",
 
 Review comment:
   good question.
   I used Gson to be consistent w/ how we do things in the healthcare example docs and a CDAP plugin that was developed for the same purpose.
   It does not seem strictly necessary.
   I will prepare a commit cacad23 to remove gson dep and migrate to jackson.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] brianlucier commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
brianlucier commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r398588185
 
 

 ##########
 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) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = filter;
+    }
+
+    /**
+     * Instantiates a new List HL7v2 message IDs without filter.
+     *
+     * @param hl7v2Stores the HL7v2 stores
+     */
+    ListHL7v2MessageIDs(List<String> hl7v2Stores) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = null;
+    }
+
+    @Override
+    public PCollection<String> expand(PBegin input) {
+      return input.apply(Create.of(this.hl7v2Stores)).apply(ParDo.of(new ListHL7v2Fn(this.filter)));
+    }
+  }
+
+  /** The type List HL7v2 fn. */
+  static class ListHL7v2Fn extends DoFn<String, String> {
+
+    private final String filter;
+    private transient HealthcareApiClient client;
+
+    /**
+     * Instantiates a new List HL7v2 fn.
+     *
+     * @param filter the filter
+     */
+    ListHL7v2Fn(String filter) {
+      this.filter = filter;
+    }
+
+    /**
+     * Init client.
+     *
+     * @throws IOException the io exception
+     */
+    @Setup
+    public void initClient() throws IOException {
+      this.client = new HttpHealthcareApiClient();
+    }
+
+    /**
+     * List messages.
+     *
+     * @param context the context
+     * @throws IOException the io exception
+     */
+    @ProcessElement
+    public void listMessages(ProcessContext context) throws IOException {
+      String hl7v2Store = context.element();
+      // Output all elements of all pages.
+      this.client.getHL7v2MessageIDStream(hl7v2Store, this.filter).forEach(context::output);
+    }
+  }
+
+  /** The type Write. */
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<HL7v2Message>, Write.Result> {
+
+    /** The tag for the successful writes to HL7v2 store`. */
+    public static final TupleTag<HealthcareIOError<HL7v2Message>> SUCCESS =
+        new TupleTag<HealthcareIOError<HL7v2Message>>() {};
+    /** The tag for the failed writes to HL7v2 store`. */
+    public static final TupleTag<HealthcareIOError<HL7v2Message>> FAILED =
+        new TupleTag<HealthcareIOError<HL7v2Message>>() {};
+
+    /**
+     * Gets HL7v2 store.
+     *
+     * @return the HL7v2 store
+     */
+    abstract String getHL7v2Store();
+
+    /**
+     * Gets write method.
+     *
+     * @return the write method
+     */
+    abstract WriteMethod getWriteMethod();
+
+    @Override
+    public Result expand(PCollection<HL7v2Message> messages) {
+      return messages.apply(new WriteHL7v2(this.getHL7v2Store(), this.getWriteMethod()));
+    }
+
+    /** The enum Write method. */
+    public enum WriteMethod {
+      /**
+       * Ingest write method. @see <a
+       * href=https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/ingest></a>
+       */
+      INGEST,
+      /**
+       * Batch import write method. This is not yet supported by the HL7v2 API, but can be used to
+       * improve throughput once available.
+       */
+      BATCH_IMPORT
+    }
+
+    /** The type Builder. */
+    @AutoValue.Builder
+    abstract static class Builder {
+
+      /**
+       * Sets HL7v2 store.
+       *
+       * @param hl7v2Store the HL7v2 store
+       * @return the HL7v2 store
+       */
+      abstract Builder setHL7v2Store(String hl7v2Store);
+
+      /**
+       * Sets write method.
+       *
+       * @param writeMethod the write method
+       * @return the write method
+       */
+      abstract Builder setWriteMethod(WriteMethod writeMethod);
+
+      /**
+       * Build write.
+       *
+       * @return the write
+       */
+      abstract Write build();
+    }
+
+    public static class Result implements POutput {
+      private final Pipeline pipeline;
+      private final PCollection<HealthcareIOError<HL7v2Message>> failedInsertsWithErr;
+
+      /** Creates a {@link HL7v2IO.Write.Result} in the given {@link Pipeline}. */
+      static Result in(
+          Pipeline pipeline, PCollection<HealthcareIOError<HL7v2Message>> failedInserts) {
+        return new Result(pipeline, failedInserts);
+      }
+
+      public PCollection<HealthcareIOError<HL7v2Message>> getFailedInsertsWithErr() {
+        return this.failedInsertsWithErr;
+      }
+
+      @Override
+      public Pipeline getPipeline() {
+        return this.pipeline;
+      }
+
+      @Override
+      public Map<TupleTag<?>, PValue> expand() {
+        failedInsertsWithErr.setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder()));
+        return ImmutableMap.of(FAILED, failedInsertsWithErr);
+      }
+
+      @Override
+      public void finishSpecifyingOutput(
+          String transformName, PInput input, PTransform<?, ?> transform) {}
+
+      private Result(
+          Pipeline pipeline, PCollection<HealthcareIOError<HL7v2Message>> failedInsertsWithErr) {
+        this.pipeline = pipeline;
+        this.failedInsertsWithErr = failedInsertsWithErr;
+      }
+    }
+  }
+
+  /** The type Write hl 7 v 2. */
+  static class WriteHL7v2 extends PTransform<PCollection<HL7v2Message>, Write.Result> {
+    private final String hl7v2Store;
+    private final Write.WriteMethod writeMethod;
+
+    /**
+     * Instantiates a new Write hl 7 v 2.
+     *
+     * @param hl7v2Store the hl 7 v 2 store
+     * @param writeMethod the write method
+     */
+    WriteHL7v2(String hl7v2Store, Write.WriteMethod writeMethod) {
+      this.hl7v2Store = hl7v2Store;
+      this.writeMethod = writeMethod;
+    }
+
+    @Override
+    public Write.Result expand(PCollection<HL7v2Message> input) {
+      PCollection<HealthcareIOError<HL7v2Message>> failedInserts =
+          input
+              .apply(ParDo.of(new WriteHL7v2Fn(hl7v2Store, writeMethod)))
+              .setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder()));
+      return Write.Result.in(input.getPipeline(), failedInserts);
+    }
+
+    /** The type Write hl 7 v 2 fn. */
+    static class WriteHL7v2Fn extends DoFn<HL7v2Message, HealthcareIOError<HL7v2Message>> {
+      // TODO when the healthcare API releases a bulk import method this should use that to improve
+      // throughput.
+
+      private Counter failedMessageWrites =
+          Metrics.counter(WriteHL7v2Fn.class, "failed-hl7v2-message-writes");
+      private final String hl7v2Store;
+      private final Counter throttledSeconds =
+          Metrics.counter(WriteHL7v2Fn.class, "cumulative-throttling-seconds");
+      private final Counter successfulHL7v2MessageWrites =
+          Metrics.counter(WriteHL7v2.class, "successful-hl7v2-message-writes");
+      private transient AdaptiveThrottler throttler;
+      private final Write.WriteMethod writeMethod;
+
+      private static final Logger LOG = LoggerFactory.getLogger(WriteHL7v2.WriteHL7v2Fn.class);
+      private transient HealthcareApiClient client;
+
+      /**
+       * Instantiates a new Write HL7v2 fn.
+       *
+       * @param hl7v2Store the HL7v2 store
+       * @param writeMethod the write method
+       */
+      WriteHL7v2Fn(String hl7v2Store, Write.WriteMethod writeMethod) {
+        this.hl7v2Store = hl7v2Store;
+        this.writeMethod = writeMethod;
+      }
+
+      /**
+       * Init client.
+       *
+       * @throws IOException the io exception
+       */
+      @Setup
+      public void initClient() throws IOException {
+        this.client = new HttpHealthcareApiClient();
+      }
+
+      @StartBundle
+      public void startBundle() {
+        if (throttler == null) {
+          throttler = new AdaptiveThrottler(1200000, 10000, 1.25);
+        }
+      }
+
+      /**
+       * Write messages.
+       *
+       * @param context the context
+       */
+      @ProcessElement
+      public void writeMessages(ProcessContext context) {
+        HL7v2Message msg = context.element();
+        final int throttleWaitSeconds = 5;
+        long startTime = System.currentTimeMillis();
+        Sleeper sleeper = Sleeper.DEFAULT;
+        switch (writeMethod) {
+          case BATCH_IMPORT:
+            throw new UnsupportedOperationException("The Batch import API is not available yet");
+          case INGEST:
+          default:
+            try {
+              if (throttler.throttleRequest(startTime)) {
+                LOG.info("Delaying request due to previous failures.");
+                this.throttledSeconds.inc(throttleWaitSeconds);
+                sleeper.sleep(throttleWaitSeconds * 1000);
+                this.throttler.successfulRequest(startTime);
+                this.successfulHL7v2MessageWrites.inc();
+              }
+              client.ingestHL7v2Message(hl7v2Store, msg.toModel());
+            } catch (Exception e) {
+              failedMessageWrites.inc();
 
 Review comment:
   I believe the reason for failure may be relevant.  a 429 would want a certain number of retries, where a 401 or 403 may need to refresh the credentials, where another error is likely a hard stop.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-611213197
 
 
   Java PostCommit is probably our slowest suite of all, but once it runs we should be good to go

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-615376750
 
 
   @chamikaramj looking into this now.
   Struggling to reproduce locally. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-604655037
 
 
   Ok an updates here from an internal thread w/ API team.
   
   1. [Message.List returning message contents is available in beta API with the view parameter.
   1. Schematized Data should be in next beta release roughly in ~2 weeks.
   1. right now the sink is outputting schematized data json wrapped in "{data=<actual_valid_json>}" 
   
   In light of these I will do the following refactors: 
   1. [x] how we batch read from to always avoid the double get. This will make it a completely parallel code path than the real-time path but I think that's ok.
   1. [x] refactor to use beta client library (once it includes schematizedData)
       - FYI pre-work for v1beta1 migration in this branch [migration/v1beta1](https://github.com/jaketf/beam/tree/migration/v1beta1). Will merge back to this branch once we have schematizedData in beta client library.
   1. [x] I'll strip out that `{data=}` wrapper to make this easier for users.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] suztomo commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
suztomo commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r409055630
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java
 ##########
 @@ -0,0 +1,459 @@
+/*
+ * 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.client.http.HttpHeaders;
+import com.google.api.client.http.HttpRequest;
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.client.http.javanet.NetHttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.healthcare.v1beta1.CloudHealthcare;
+import com.google.api.services.healthcare.v1beta1.CloudHealthcare.Projects.Locations.Datasets.Hl7V2Stores.Messages;
+import com.google.api.services.healthcare.v1beta1.CloudHealthcareScopes;
+import com.google.api.services.healthcare.v1beta1.model.CreateMessageRequest;
+import com.google.api.services.healthcare.v1beta1.model.Empty;
+import com.google.api.services.healthcare.v1beta1.model.Hl7V2Store;
+import com.google.api.services.healthcare.v1beta1.model.HttpBody;
+import com.google.api.services.healthcare.v1beta1.model.IngestMessageRequest;
+import com.google.api.services.healthcare.v1beta1.model.IngestMessageResponse;
+import com.google.api.services.healthcare.v1beta1.model.ListMessagesResponse;
+import com.google.api.services.healthcare.v1beta1.model.Message;
+import com.google.api.services.healthcare.v1beta1.model.SearchResourcesRequest;
+import com.google.auth.oauth2.GoogleCredentials;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URI;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Spliterator;
+import java.util.Spliterators;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.gcp.util.RetryHttpRequestInitializer;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
+
+/**
+ * A client that talks to the Cloud Healthcare API through HTTP requests. This client is created
+ * mainly to encapsulate the unserializable dependencies, since most generated classes are not
+ * serializable in the HTTP client.
+ */
+public class HttpHealthcareApiClient<T> implements HealthcareApiClient, Serializable {
+
+  private transient CloudHealthcare client;
+
+  /**
+   * Instantiates a new Http healthcare api client.
+   *
+   * @throws IOException the io exception
+   */
+  public HttpHealthcareApiClient() throws IOException {
+    initClient();
+  }
+
+  /**
+   * Instantiates a new Http healthcare api client.
+   *
+   * @param client the client
+   * @throws IOException the io exception
+   */
+  public HttpHealthcareApiClient(CloudHealthcare client) throws IOException {
+    this.client = client;
+    initClient();
+  }
+
+  @VisibleForTesting
+  static <T, X extends Collection<T>> Stream<T> flattenIteratorCollectionsToStream(
+      Iterator<X> iterator) {
+    Spliterator<Collection<T>> spliterator = Spliterators.spliteratorUnknownSize(iterator, 0);
+    return StreamSupport.stream(spliterator, false).flatMap(Collection::stream);
+  }
+
+  public JsonFactory getJsonFactory() {
+    return this.client.getJsonFactory();
+  }
+
+  @Override
+  public Hl7V2Store createHL7v2Store(String dataset, String name) throws IOException {
+    Hl7V2Store store = new Hl7V2Store();
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .create(dataset, store)
+        .setHl7V2StoreId(name)
+        .execute();
+  }
+
+  @Override
+  public Empty deleteHL7v2Store(String name) throws IOException {
+    return client.projects().locations().datasets().hl7V2Stores().delete(name).execute();
+  }
+
+  @Override
+  public ListMessagesResponse makeHL7v2ListRequest(
+      String hl7v2Store, @Nullable String filter, @Nullable String pageToken) throws IOException {
+
+    Messages.List baseRequest =
+        client
+            .projects()
+            .locations()
+            .datasets()
+            .hl7V2Stores()
+            .messages()
+            .list(hl7v2Store)
+            .set("view", "full")
+            .setPageToken(pageToken);
+
+    if (Strings.isNullOrEmpty(filter)) {
+      return baseRequest.execute();
+    } else {
+      return baseRequest.setFilter(filter).execute();
+    }
+  }
+
+  /**
+   * Gets message id page iterator.
+   *
+   * @param hl7v2Store the HL7v2 store
+   * @return the message id page iterator
+   * @throws IOException the io exception
+   */
+  @Override
+  public Stream<HL7v2Message> getHL7v2MessageStream(String hl7v2Store) throws IOException {
+    return getHL7v2MessageStream(hl7v2Store, null);
+  }
+
+  /**
+   * Get a {@link Stream} of message IDs from flattening the pages of a new {@link
+   * HL7v2MessagePages}.
+   *
+   * @param hl7v2Store the HL7v2 store
+   * @param filter the filter
+   * @return the message id Stream
+   * @throws IOException the io exception
+   */
+  @Override
+  public Stream<HL7v2Message> getHL7v2MessageStream(String hl7v2Store, @Nullable String filter)
+      throws IOException {
+    Iterator<List<HL7v2Message>> iterator =
+        new HL7v2MessagePages(this, hl7v2Store, filter).iterator();
+    return flattenIteratorCollectionsToStream(iterator);
+  }
+
+  /**
+   * Gets HL7v2 message.
+   *
+   * @param msgName the msg name
+   * @return the message
+   * @throws IOException the io exception
+   * @throws ParseException the parse exception
+   */
+  @Override
+  public Message getHL7v2Message(String msgName) throws IOException {
+    Message msg =
+        client.projects().locations().datasets().hl7V2Stores().messages().get(msgName).execute();
+    if (msg == null) {
+      throw new IOException(String.format("Couldn't find message: %s.", msgName));
+    }
+    return msg;
+  }
+
+  @Override
+  public Empty deleteHL7v2Message(String msgName) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .delete(msgName)
+        .execute();
+  }
+
+  /**
+   * Gets HL7v2 store.
+   *
+   * @param storeName the store name
+   * @return the HL7v2 store
+   * @throws IOException the io exception
+   */
+  @Override
+  public Hl7V2Store getHL7v2Store(String storeName) throws IOException {
+    return client.projects().locations().datasets().hl7V2Stores().get(storeName).execute();
+  }
+
+  @Override
+  public IngestMessageResponse ingestHL7v2Message(String hl7v2Store, Message msg)
+      throws IOException {
+    IngestMessageRequest ingestMessageRequest = new IngestMessageRequest();
+    ingestMessageRequest.setMessage(msg);
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .ingest(hl7v2Store, ingestMessageRequest)
+        .execute();
+  }
+
+  @Override
+  public HttpBody fhirSearch(String fhirStore, SearchResourcesRequest query) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .search(fhirStore, query)
+        .execute();
+  }
+
+  @Override
+  public Message createHL7v2Message(String hl7v2Store, Message msg) throws IOException {
+    CreateMessageRequest createMessageRequest = new CreateMessageRequest();
+    createMessageRequest.setMessage(msg);
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .create(hl7v2Store, createMessageRequest)
+        .execute();
+  }
+
+  @Override
+  public HttpBody createFhirResource(String fhirStore, String type, HttpBody body)
+      throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .create(fhirStore, type, body)
+        .execute();
+  }
+
+  @Override
+  public HttpBody executeFhirBundle(String fhirStore, HttpBody bundle) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .executeBundle(fhirStore, bundle)
+        .execute();
+  }
+
+  @Override
+  public HttpBody listFHIRResourceForPatient(String fhirStore, String patient) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .patientEverything(patient)
+        .execute();
+  }
+
+  @Override
+  public HttpBody readFHIRResource(String fhirStore, String resource) throws IOException {
+    return client.projects().locations().datasets().fhirStores().fhir().read(resource).execute();
+  }
+
+  private static class AuthenticatedRetryInitializer extends RetryHttpRequestInitializer {
+    GoogleCredentials credentials;
+
+    public AuthenticatedRetryInitializer(GoogleCredentials credentials) {
+      super();
+      this.credentials = credentials;
+    }
+
+    @Override
+    public void initialize(HttpRequest request) throws IOException {
+      super.initialize(request);
+      if (!credentials.hasRequestMetadata()) {
+        return;
+      }
+      HttpHeaders requestHeaders = request.getHeaders();
+      requestHeaders.setUserAgent("apache-beam-hl7v2-io");
+      URI uri = null;
+      if (request.getUrl() != null) {
+        uri = request.getUrl().toURI();
+      }
+      Map<String, List<String>> credentialHeaders = credentials.getRequestMetadata(uri);
+      if (credentialHeaders == null) {
+        return;
+      }
+      for (Map.Entry<String, List<String>> entry : credentialHeaders.entrySet()) {
+        String headerName = entry.getKey();
+        List<String> requestValues = new ArrayList<>(entry.getValue());
+        requestHeaders.put(headerName, requestValues);
+      }
+    }
+  }
+
+  private void initClient() throws IOException {
+    // Create a HttpRequestInitializer, which will provide a baseline configuration to all requests.
+    // HttpRequestInitializer requestInitializer = new RetryHttpRequestInitializer();
+    // GoogleCredentials credentials = GoogleCredentials.getApplicationDefault();
+    HttpRequestInitializer requestInitializer =
+        new AuthenticatedRetryInitializer(
+            GoogleCredentials.getApplicationDefault()
+                .createScoped(CloudHealthcareScopes.CLOUD_PLATFORM));
+
+    client =
+        new CloudHealthcare.Builder(new NetHttpTransport(), new GsonFactory(), requestInitializer)
 
 Review comment:
   Memo: this uses `com.google.api.client.json.gson.GsonFactory`.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-603993805
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612179469
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614238420
 
 
   retest this please
   
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612327959
 
 
   Run Java PreCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-609987600
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614240401
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-600890728
 
 
   @lastomato
   Thoughts on how we could rework the HCAPI client to get around this failing CI Check?
   https://builds.apache.org/job/beam_PreCommit_Java_Commit/10418/testReport/org.apache.beam.sdk.io.gcp/GcpApiSurfaceTest/testGcpApiSurface/

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] brianlucier commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
brianlucier commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r398586106
 
 

 ##########
 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) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = filter;
+    }
+
+    /**
+     * Instantiates a new List HL7v2 message IDs without filter.
+     *
+     * @param hl7v2Stores the HL7v2 stores
+     */
+    ListHL7v2MessageIDs(List<String> hl7v2Stores) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = null;
+    }
+
+    @Override
+    public PCollection<String> expand(PBegin input) {
+      return input.apply(Create.of(this.hl7v2Stores)).apply(ParDo.of(new ListHL7v2Fn(this.filter)));
+    }
+  }
+
+  /** The type List HL7v2 fn. */
+  static class ListHL7v2Fn extends DoFn<String, String> {
+
+    private final String filter;
+    private transient HealthcareApiClient client;
+
+    /**
+     * Instantiates a new List HL7v2 fn.
+     *
+     * @param filter the filter
+     */
+    ListHL7v2Fn(String filter) {
+      this.filter = filter;
+    }
+
+    /**
+     * Init client.
+     *
+     * @throws IOException the io exception
+     */
+    @Setup
+    public void initClient() throws IOException {
+      this.client = new HttpHealthcareApiClient();
+    }
+
+    /**
+     * List messages.
+     *
+     * @param context the context
+     * @throws IOException the io exception
+     */
+    @ProcessElement
+    public void listMessages(ProcessContext context) throws IOException {
+      String hl7v2Store = context.element();
+      // Output all elements of all pages.
+      this.client.getHL7v2MessageIDStream(hl7v2Store, this.filter).forEach(context::output);
+    }
+  }
+
+  /** The type Write. */
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<HL7v2Message>, Write.Result> {
+
+    /** The tag for the successful writes to HL7v2 store`. */
+    public static final TupleTag<HealthcareIOError<HL7v2Message>> SUCCESS =
+        new TupleTag<HealthcareIOError<HL7v2Message>>() {};
+    /** The tag for the failed writes to HL7v2 store`. */
+    public static final TupleTag<HealthcareIOError<HL7v2Message>> FAILED =
+        new TupleTag<HealthcareIOError<HL7v2Message>>() {};
+
+    /**
+     * Gets HL7v2 store.
+     *
+     * @return the HL7v2 store
+     */
+    abstract String getHL7v2Store();
+
+    /**
+     * Gets write method.
+     *
+     * @return the write method
+     */
+    abstract WriteMethod getWriteMethod();
+
+    @Override
+    public Result expand(PCollection<HL7v2Message> messages) {
+      return messages.apply(new WriteHL7v2(this.getHL7v2Store(), this.getWriteMethod()));
+    }
+
+    /** The enum Write method. */
+    public enum WriteMethod {
+      /**
+       * Ingest write method. @see <a
+       * href=https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/ingest></a>
+       */
+      INGEST,
+      /**
+       * Batch import write method. This is not yet supported by the HL7v2 API, but can be used to
+       * improve throughput once available.
+       */
+      BATCH_IMPORT
+    }
+
+    /** The type Builder. */
+    @AutoValue.Builder
+    abstract static class Builder {
+
+      /**
+       * Sets HL7v2 store.
+       *
+       * @param hl7v2Store the HL7v2 store
+       * @return the HL7v2 store
+       */
+      abstract Builder setHL7v2Store(String hl7v2Store);
+
+      /**
+       * Sets write method.
+       *
+       * @param writeMethod the write method
+       * @return the write method
+       */
+      abstract Builder setWriteMethod(WriteMethod writeMethod);
+
+      /**
+       * Build write.
+       *
+       * @return the write
+       */
+      abstract Write build();
+    }
+
+    public static class Result implements POutput {
+      private final Pipeline pipeline;
+      private final PCollection<HealthcareIOError<HL7v2Message>> failedInsertsWithErr;
+
+      /** Creates a {@link HL7v2IO.Write.Result} in the given {@link Pipeline}. */
+      static Result in(
+          Pipeline pipeline, PCollection<HealthcareIOError<HL7v2Message>> failedInserts) {
+        return new Result(pipeline, failedInserts);
+      }
+
+      public PCollection<HealthcareIOError<HL7v2Message>> getFailedInsertsWithErr() {
+        return this.failedInsertsWithErr;
+      }
+
+      @Override
+      public Pipeline getPipeline() {
+        return this.pipeline;
+      }
+
+      @Override
+      public Map<TupleTag<?>, PValue> expand() {
+        failedInsertsWithErr.setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder()));
+        return ImmutableMap.of(FAILED, failedInsertsWithErr);
+      }
+
+      @Override
+      public void finishSpecifyingOutput(
+          String transformName, PInput input, PTransform<?, ?> transform) {}
+
+      private Result(
+          Pipeline pipeline, PCollection<HealthcareIOError<HL7v2Message>> failedInsertsWithErr) {
+        this.pipeline = pipeline;
+        this.failedInsertsWithErr = failedInsertsWithErr;
+      }
+    }
+  }
+
+  /** The type Write hl 7 v 2. */
+  static class WriteHL7v2 extends PTransform<PCollection<HL7v2Message>, Write.Result> {
+    private final String hl7v2Store;
+    private final Write.WriteMethod writeMethod;
+
+    /**
+     * Instantiates a new Write hl 7 v 2.
+     *
+     * @param hl7v2Store the hl 7 v 2 store
+     * @param writeMethod the write method
+     */
+    WriteHL7v2(String hl7v2Store, Write.WriteMethod writeMethod) {
+      this.hl7v2Store = hl7v2Store;
+      this.writeMethod = writeMethod;
+    }
+
+    @Override
+    public Write.Result expand(PCollection<HL7v2Message> input) {
+      PCollection<HealthcareIOError<HL7v2Message>> failedInserts =
+          input
+              .apply(ParDo.of(new WriteHL7v2Fn(hl7v2Store, writeMethod)))
+              .setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder()));
+      return Write.Result.in(input.getPipeline(), failedInserts);
+    }
+
+    /** The type Write hl 7 v 2 fn. */
+    static class WriteHL7v2Fn extends DoFn<HL7v2Message, HealthcareIOError<HL7v2Message>> {
+      // TODO when the healthcare API releases a bulk import method this should use that to improve
+      // throughput.
+
+      private Counter failedMessageWrites =
+          Metrics.counter(WriteHL7v2Fn.class, "failed-hl7v2-message-writes");
+      private final String hl7v2Store;
+      private final Counter throttledSeconds =
+          Metrics.counter(WriteHL7v2Fn.class, "cumulative-throttling-seconds");
+      private final Counter successfulHL7v2MessageWrites =
+          Metrics.counter(WriteHL7v2.class, "successful-hl7v2-message-writes");
+      private transient AdaptiveThrottler throttler;
+      private final Write.WriteMethod writeMethod;
+
+      private static final Logger LOG = LoggerFactory.getLogger(WriteHL7v2.WriteHL7v2Fn.class);
+      private transient HealthcareApiClient client;
+
+      /**
+       * Instantiates a new Write HL7v2 fn.
+       *
+       * @param hl7v2Store the HL7v2 store
+       * @param writeMethod the write method
+       */
+      WriteHL7v2Fn(String hl7v2Store, Write.WriteMethod writeMethod) {
+        this.hl7v2Store = hl7v2Store;
+        this.writeMethod = writeMethod;
+      }
+
+      /**
+       * Init client.
+       *
+       * @throws IOException the io exception
+       */
+      @Setup
+      public void initClient() throws IOException {
+        this.client = new HttpHealthcareApiClient();
+      }
+
+      @StartBundle
+      public void startBundle() {
+        if (throttler == null) {
+          throttler = new AdaptiveThrottler(1200000, 10000, 1.25);
+        }
+      }
+
+      /**
+       * Write messages.
+       *
+       * @param context the context
+       */
+      @ProcessElement
+      public void writeMessages(ProcessContext context) {
+        HL7v2Message msg = context.element();
+        final int throttleWaitSeconds = 5;
+        long startTime = System.currentTimeMillis();
+        Sleeper sleeper = Sleeper.DEFAULT;
+        switch (writeMethod) {
+          case BATCH_IMPORT:
+            throw new UnsupportedOperationException("The Batch import API is not available yet");
 
 Review comment:
   I like that the switch is here... looking fwd to it :)

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613078385
 
 
   oops. looks like a merge issue
   ```
   13:11:33 /home/jenkins/jenkins-slave/workspace/beam_PreCommit_Java_Commit/src/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy: 386: The current scope already contains a variable of the name google_oauth_clients_version
   13:11:33  @ line 386, column 9.
   13:11:33        def google_oauth_clients_version = "1.30.6"
   13:11:33            ^
   13:11:33 
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-600786687
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r406855406
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   https://cwiki.apache.org/confluence/display/BEAM/Dependency+Upgrades
   
   Combining the linkage checker with a bunch of precommits and postcommits gives us the confidence that the change is safe.
   
   The linkage checker is a tool which finds linkage errors by scanning the classpath and finding missing classes/methods. You run the linkage checker on all modules before and after your change and show that the linkage errors reduced or stayed the same. Any new linkage errors need to be followed up and explained why they are safe.
   
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-610050634
 
 
   Next Steps (based on offline feedback):
   - [x] Improve API for users:
       - [x] Add static methods for common patterns with `ListHL7v2Messages` 
       - [x] Add `ValueProvider<String>` support to ease use in the DataflowTemplates
           - [x] `ListHL7v2Messages` (hl7v2Store and filter)
           - [x] `Write` (hl7v2Store)
   - [x] "standardize" integration tests
       - [x] Refactor ITs to create / destroy HL7v2 Store under a parameterized dataset in `@BeforeClass` `@AfterClass` to avoid issues with parallel tests runs.
       - [x] Remove hard coding of my HL7v2Store / project in integration tests.
       - [x] Add Healthcare API Dataset to Beam integration test project (pending permissions in [this dev list thread](https://lists.apache.org/thread.html/rebe5cd40a40a9fc7f2c1d563b48ee1ce4ff9cac3dfdc0258006cc686%40%3Cdev.beam.apache.org%3E))

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-611081036
 
 
   @pabloem 
   Questions:
   
   1. How could / should we add this to this [page for built in connectors](https://beam.apache.org/documentation/io/built-in/)? Are there other documentation things that I might have missed? 
   1. Should we file jira issues or buganizer  for future work, notably:
       - Using Batch import / export for read when that api is available
       - Updating to newer version of client library (coming soon, which will simplify the hack-y handling of schematized data in this PR)
   1. What is the feasibility / process for back-porting this to prior beam releases for expediting use by users who want to use an already released version e.g. 2.19 ? 
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-610704882
 
 
   @pabloem PTAL.
   I've addressed all outstanding feedback and setup integration testing infrastructure
   
   Note on integration tests:
   Integration tests seem to be passing.
   Created the following resources
   - [`gs://temp-storage-for-healthcare-io-tests`](https://pantheon.corp.google.com/storage/browser/temp-storage-for-healthcare-io-tests/?forceOnBucketsSortingFiltering=false&project=apache-beam-testing)
       - will house temp files for FHIR Import in upcoming FhirIO PR
   - [`projects/apache-beam-testing/locations/us-central1/datasets/apache-beam-integration-testing`](https://pantheon.corp.google.com/healthcare/browser/locations/us-central1/datasets/apache-beam-integration-testing/datastores?project=apache-beam-testing) which houses ephemeral HL7v2 store per test class (and later ephemeral FHIR store per class.)

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-600748606
 
 
   R: @pabloem 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-603993940
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r406327077
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   Good question!
   Please let me know if I'm misunderstanding but I believe that is [set to 1.30.3](https://github.com/apache/beam/blob/master/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy#L379)
   And looking at this [repo for healthcare client](https://github.com/googleapis/google-api-java-client-services/tree/master/clients/google-api-services-healthcare/v1beta1) there doesn't seem to be anything past 1.30.1 (which was recently regenerated).
   
   Also FYI you're commenting on an outdated commit we are now using beta client lib.
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612278707
 
 
   Run Java PreCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-600378956
 
 
   @lastomato FYI Moved this to it's own PR.
   Once we get feedback and iterate on this with beam reviewer, I can open a PR for FHIRIO.
   
   
   
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-603992173
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404362779
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOWriteIT.java
 ##########
 @@ -0,0 +1,72 @@
+/*
+ * 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 static org.apache.beam.sdk.io.gcp.healthcare.HL7v2IOTestUtil.MESSAGES;
+import static org.apache.beam.sdk.io.gcp.healthcare.HL7v2IOTestUtil.deleteAllHL7v2Messages;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class HL7v2IOWriteIT {
+
+  private HL7v2IOTestOptions options;
+  private transient HealthcareApiClient client;
+
+  @Before
+  public void setup() throws Exception {
+    if (client == null) {
+      client = new HttpHealthcareApiClient();
+    }
+    PipelineOptionsFactory.register(HL7v2IOTestOptions.class);
+    options = TestPipeline.testingPipelineOptions().as(HL7v2IOTestOptions.class);
+    options.setHL7v2Store(
+        "projects/jferriero-dev/locations/us-central1/datasets/raw-dataset/hl7V2Stores/jake-hl7");
 
 Review comment:
   Yes it does write to HL7v2 store.
   These Integration Tests should not run in parallel (as the success criteria is based on counting messages in HL7v2 store).
   
   This should be very low maintenance. Let's connect offline about how you typically create resources in this project.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-609987751
 
 
   I'm starting to take a good look at this now.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613060312
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404374442
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,597 @@
+/*
+ * 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.v1beta1.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.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 can be fetched from the HL7v2 store in two ways Message Fetching and Message
+ * Listing.
+ *
+ * <p>Message Fetching
+ *
+ * <p>Message Fetching with {@link HL7v2IO.Read} supports use cases where you have a ${@link
+ * PCollection} of message IDS. This is appropriate for reading the HL7v2 notifications from a
+ * Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually
+ * prepared list of messages that you need to process (e.g. in a text file read with {@link
+ * org.apache.beam.sdk.io.TextIO}) .
+ *
+ * <p>Fetch Message contents from HL7v2 Store based on the {@link PCollection} of message ID strings
+ * {@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},
+ * this can be used to write to the dead letter storage system of your choosing. This error handling
+ * is mainly to catch scenarios where the upstream {@link PCollection} contains IDs that are not
+ * valid or are not reachable due to permissions issues.
+ *
+ * <p>Message Listing Message Listing with {@link HL7v2IO.ListHL7v2Messages} supports batch use
+ * cases where you want to process all the messages in an HL7v2 store or those matching a
+ * filter @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters</a>
+ * This paginates through results of a Messages.List call @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list</a>
+ * and outputs directly to a {@link PCollection} of {@link HL7v2Message}. In these use cases, the
+ * error handling similar to above is unnecessary because we are listing from the source of truth
+ * the pipeline should fail transparently if this transform fails to paginate through all the
+ * results.
+ *
+ * <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} containing the {@link HL7v2Message}
+ * 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 Read Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * PCollection<HL7v2Message> out = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2Messages(
+ *           Collections.singletonList(options.getInputHL7v2Store()), option.getHL7v2Filter()))
+ *    // Go about your happy path transformations.
+ *   .apply("Process HL7v2 Messages", ...);
+ * 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 ListHL7v2Messages} 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} message IDs which failed to be fetched from the HL7v2 store, with
+     *       error message and stacktrace.
+     * </ul>
+     */
+    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 successfulHL7v2MessageGets =
+            Metrics.counter(
+                FetchHL7v2Message.HL7v2MessageGetFn.class, "successful-hl7v2-message-gets");
+        private HealthcareApiClient client;
+
+        /** 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();
+        }
+
+        /**
+         * 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 {
+          long startTime = System.currentTimeMillis();
+
+          com.google.api.services.healthcare.v1beta1.model.Message msg =
+              client.getHL7v2Message(msgId);
+
+          if (msg == null) {
+            throw new IOException(String.format("GET request for %s returned null", msgId));
+          }
+          this.successfulHL7v2MessageGets.inc();
+          return msg;
+        }
+      }
+    }
+  }
+
+  /** List HL7v2 messages. */
+  public static class ListHL7v2Messages extends PTransform<PBegin, PCollection<HL7v2Message>> {
+    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
+     */
+    ListHL7v2Messages(List<String> hl7v2Stores, String filter) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = filter;
+    }
+
+    ListHL7v2Messages(List<String> hl7v2Stores) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = null;
+    }
+
+    @Override
+    public PCollection<HL7v2Message> expand(PBegin input) {
+      return input
+          .apply(Create.of(this.hl7v2Stores))
+          .apply(ParDo.of(new ListHL7v2MessagesFn(this.filter)));
+    }
+  }
+
+  static class ListHL7v2MessagesFn extends DoFn<String, HL7v2Message> {
+
+    private final String filter;
+    private transient HealthcareApiClient client;
+
+    /**
+     * Instantiates a new List HL7v2 fn.
+     *
+     * @param filter the filter
+     */
+    ListHL7v2MessagesFn(String filter) {
+      this.filter = filter;
+    }
+
+    /**
+     * Init client.
+     *
+     * @throws IOException the io exception
+     */
+    @Setup
+    public void initClient() throws IOException {
+      this.client = new HttpHealthcareApiClient();
+    }
+
+    /**
+     * List messages.
+     *
+     * @param context the context
+     * @throws IOException the io exception
+     */
+    @ProcessElement
+    public void listMessages(ProcessContext context) throws IOException {
+      String hl7v2Store = context.element();
+      // Output all elements of all pages.
+      this.client.getHL7v2MessageStream(hl7v2Store, this.filter).forEach(context::output);
 
 Review comment:
   The message listing implementation in `HttpHealthcareApiClient.HL7v2MessagePages` iterator will be a bottleneck here as it has to paginate through the list results in a single thread. The API does not support reading from an offset.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r394668514
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,620 @@
+/*
+ * 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.IngestMessageResponse;
+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.Collections;
+import java.util.List;
+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.PDone;
+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.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 HL7v2 Messages are fetched from the HL7v2 store based on the {@link PCollection} of of
+ * message IDs {@link String}s produced by the {@link AutoValue_HL7v2IO_Read#getMessageIDTransform}
+ * as {@link PCollectionTuple}*** containing an {@link HL7v2IO.Read#OUT} tag for successfully
+ * fetched messages and a {@link HL7v2IO.Read#DEAD_LETTER} tag for message IDs that could not be
+ * fetched.
+ *
+ * <p>HL7v2 stores can be read in several ways: - Unbounded: based on the Pub/Sub Notification
+ * Channel {@link HL7v2IO#readNotificationSubscription(String)} - Bounded: based on reading an
+ * entire HL7v2 store (or stores) {@link HL7v2IO#readHL7v2Store(String)} - Bounded: based on reading
+ * an HL7v2 store with a filter
+ *
+ * <p>Note, due to the flexibility of this Read transform, this must output a dead letter queue.
+ * This handles the scenario where the the PTransform that populates a PCollection of message IDs
+ * contains message IDs that do not exist in the HL7v2 stores.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline pipeline = Pipeline.create(options)
+ *
+ *
+ * PCollectionTuple messages = pipeline.apply(
+ *     new HLv2IO.readNotifications(options.getNotificationSubscription())
+ *
+ * // Write errors to your favorite dead letter  queue (e.g. Pub/Sub, GCS, BigQuery)
+ * messages.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.
+ * PCollection<Message> out = fetchedMessages.apply("ProcessFetchedMessages", ...);
+ *
+ * // Write using the Message.Ingest method of the HL7v2 REST API.
+ * out.apply(HL7v2IO.ingestMessages(options.getOutputHL7v2Store()));
+ *
+ * pipeline.run();
+ *
+ * }***
+ * </pre>
+ */
+public class HL7v2IO {
+  // TODO add metrics for failed records.
+
+  private static Read.Builder read(PTransform<PBegin, PCollection<String>> messageIDTransform) {
 
 Review comment:
   Under this interface, you would do something like this:
   ```
   Pipeline p = ....
   PCollectionTuple pct = p.apply(HL7v2IO.read(PubSubIO.read().fromTopic(....)));
   ```
   
   This is a little odd. A more natural way of doing this would be something like this:
   
   ```
   Pipeline p = ....
   PCollection<String> messageIds = p.apply(PubSubIO.read().fromTopic(....));
   
   HL7v2IO.Read.Result result = messageIds.apply(HL7v2IO.readAll());
   ```
   
   This way, `HL7v2IO.Read` takes in a `PCollection<String>` containing the message IDs, instead of having a PTransform be a parameter, which is not commonly done elsewhere.
   
   This is similar to how FileIO transforms work. Check out the functions `match`, `matchAll`, `readMatches`. https://beam.apache.org/releases/javadoc/2.19.0/index.html?org/apache/beam/sdk/io/FileIO.html

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
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 new list API to save the extra retrieval.
   
   IMO this DoFn should remain as is to support backwards compatibility with the beta 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 old api requiring "get" after list: `HL7v2IO.ListHL7v2MessageIDs` -> `HL7v2IO.Read`(current path).
      - Add a new PTransform for new 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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-610050634
 
 
   Next Steps (based on offline feed):
   
   - [x] Improve API for users:
       - [x] Add static methods for common patterns with `ListHL7v2Messages` 
       - [x] Add `ValueProvider<String>` support to ease use in the DataflowTemplates
           - [x] `ListHL7v2Messages` (hl7v2Store and filter)
           - [x] `Write` (hl7v2Store)
   - [ ] "standardize" integration tests
       - [ ] Remove hard coding of my HL7v2Store / project in integration tests.
       - [ ] Add HL7v2 store to Beam GCP IT project

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614251554
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r398856606
 
 

 ##########
 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) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = filter;
+    }
+
+    /**
+     * Instantiates a new List HL7v2 message IDs without filter.
+     *
+     * @param hl7v2Stores the HL7v2 stores
+     */
+    ListHL7v2MessageIDs(List<String> hl7v2Stores) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = null;
+    }
+
+    @Override
+    public PCollection<String> expand(PBegin input) {
+      return input.apply(Create.of(this.hl7v2Stores)).apply(ParDo.of(new ListHL7v2Fn(this.filter)));
+    }
+  }
+
+  /** The type List HL7v2 fn. */
+  static class ListHL7v2Fn extends DoFn<String, String> {
+
+    private final String filter;
+    private transient HealthcareApiClient client;
+
+    /**
+     * Instantiates a new List HL7v2 fn.
+     *
+     * @param filter the filter
+     */
+    ListHL7v2Fn(String filter) {
+      this.filter = filter;
+    }
+
+    /**
+     * Init client.
+     *
+     * @throws IOException the io exception
+     */
+    @Setup
+    public void initClient() throws IOException {
+      this.client = new HttpHealthcareApiClient();
+    }
+
+    /**
+     * List messages.
+     *
+     * @param context the context
+     * @throws IOException the io exception
+     */
+    @ProcessElement
+    public void listMessages(ProcessContext context) throws IOException {
+      String hl7v2Store = context.element();
+      // Output all elements of all pages.
+      this.client.getHL7v2MessageIDStream(hl7v2Store, this.filter).forEach(context::output);
+    }
+  }
+
+  /** The type Write. */
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<HL7v2Message>, Write.Result> {
+
+    /** The tag for the successful writes to HL7v2 store`. */
+    public static final TupleTag<HealthcareIOError<HL7v2Message>> SUCCESS =
+        new TupleTag<HealthcareIOError<HL7v2Message>>() {};
+    /** The tag for the failed writes to HL7v2 store`. */
+    public static final TupleTag<HealthcareIOError<HL7v2Message>> FAILED =
+        new TupleTag<HealthcareIOError<HL7v2Message>>() {};
+
+    /**
+     * Gets HL7v2 store.
+     *
+     * @return the HL7v2 store
+     */
+    abstract String getHL7v2Store();
+
+    /**
+     * Gets write method.
+     *
+     * @return the write method
+     */
+    abstract WriteMethod getWriteMethod();
+
+    @Override
+    public Result expand(PCollection<HL7v2Message> messages) {
+      return messages.apply(new WriteHL7v2(this.getHL7v2Store(), this.getWriteMethod()));
+    }
+
+    /** The enum Write method. */
+    public enum WriteMethod {
+      /**
+       * Ingest write method. @see <a
+       * href=https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/ingest></a>
+       */
+      INGEST,
+      /**
+       * Batch import write method. This is not yet supported by the HL7v2 API, but can be used to
+       * improve throughput once available.
+       */
+      BATCH_IMPORT
+    }
+
+    /** The type Builder. */
+    @AutoValue.Builder
+    abstract static class Builder {
+
+      /**
+       * Sets HL7v2 store.
+       *
+       * @param hl7v2Store the HL7v2 store
+       * @return the HL7v2 store
+       */
+      abstract Builder setHL7v2Store(String hl7v2Store);
+
+      /**
+       * Sets write method.
+       *
+       * @param writeMethod the write method
+       * @return the write method
+       */
+      abstract Builder setWriteMethod(WriteMethod writeMethod);
+
+      /**
+       * Build write.
+       *
+       * @return the write
+       */
+      abstract Write build();
+    }
+
+    public static class Result implements POutput {
+      private final Pipeline pipeline;
+      private final PCollection<HealthcareIOError<HL7v2Message>> failedInsertsWithErr;
+
+      /** Creates a {@link HL7v2IO.Write.Result} in the given {@link Pipeline}. */
+      static Result in(
+          Pipeline pipeline, PCollection<HealthcareIOError<HL7v2Message>> failedInserts) {
+        return new Result(pipeline, failedInserts);
+      }
+
+      public PCollection<HealthcareIOError<HL7v2Message>> getFailedInsertsWithErr() {
+        return this.failedInsertsWithErr;
+      }
+
+      @Override
+      public Pipeline getPipeline() {
+        return this.pipeline;
+      }
+
+      @Override
+      public Map<TupleTag<?>, PValue> expand() {
+        failedInsertsWithErr.setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder()));
+        return ImmutableMap.of(FAILED, failedInsertsWithErr);
+      }
+
+      @Override
+      public void finishSpecifyingOutput(
+          String transformName, PInput input, PTransform<?, ?> transform) {}
+
+      private Result(
+          Pipeline pipeline, PCollection<HealthcareIOError<HL7v2Message>> failedInsertsWithErr) {
+        this.pipeline = pipeline;
+        this.failedInsertsWithErr = failedInsertsWithErr;
+      }
+    }
+  }
+
+  /** The type Write hl 7 v 2. */
+  static class WriteHL7v2 extends PTransform<PCollection<HL7v2Message>, Write.Result> {
+    private final String hl7v2Store;
+    private final Write.WriteMethod writeMethod;
+
+    /**
+     * Instantiates a new Write hl 7 v 2.
+     *
+     * @param hl7v2Store the hl 7 v 2 store
+     * @param writeMethod the write method
+     */
+    WriteHL7v2(String hl7v2Store, Write.WriteMethod writeMethod) {
+      this.hl7v2Store = hl7v2Store;
+      this.writeMethod = writeMethod;
+    }
+
+    @Override
+    public Write.Result expand(PCollection<HL7v2Message> input) {
+      PCollection<HealthcareIOError<HL7v2Message>> failedInserts =
+          input
+              .apply(ParDo.of(new WriteHL7v2Fn(hl7v2Store, writeMethod)))
+              .setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder()));
+      return Write.Result.in(input.getPipeline(), failedInserts);
+    }
+
+    /** The type Write hl 7 v 2 fn. */
+    static class WriteHL7v2Fn extends DoFn<HL7v2Message, HealthcareIOError<HL7v2Message>> {
+      // TODO when the healthcare API releases a bulk import method this should use that to improve
+      // throughput.
+
+      private Counter failedMessageWrites =
+          Metrics.counter(WriteHL7v2Fn.class, "failed-hl7v2-message-writes");
+      private final String hl7v2Store;
+      private final Counter throttledSeconds =
+          Metrics.counter(WriteHL7v2Fn.class, "cumulative-throttling-seconds");
+      private final Counter successfulHL7v2MessageWrites =
+          Metrics.counter(WriteHL7v2.class, "successful-hl7v2-message-writes");
+      private transient AdaptiveThrottler throttler;
+      private final Write.WriteMethod writeMethod;
+
+      private static final Logger LOG = LoggerFactory.getLogger(WriteHL7v2.WriteHL7v2Fn.class);
+      private transient HealthcareApiClient client;
+
+      /**
+       * Instantiates a new Write HL7v2 fn.
+       *
+       * @param hl7v2Store the HL7v2 store
+       * @param writeMethod the write method
+       */
+      WriteHL7v2Fn(String hl7v2Store, Write.WriteMethod writeMethod) {
+        this.hl7v2Store = hl7v2Store;
+        this.writeMethod = writeMethod;
+      }
+
+      /**
+       * Init client.
+       *
+       * @throws IOException the io exception
+       */
+      @Setup
+      public void initClient() throws IOException {
+        this.client = new HttpHealthcareApiClient();
+      }
+
+      @StartBundle
+      public void startBundle() {
+        if (throttler == null) {
+          throttler = new AdaptiveThrottler(1200000, 10000, 1.25);
+        }
+      }
+
+      /**
+       * Write messages.
+       *
+       * @param context the context
+       */
+      @ProcessElement
+      public void writeMessages(ProcessContext context) {
+        HL7v2Message msg = context.element();
+        final int throttleWaitSeconds = 5;
+        long startTime = System.currentTimeMillis();
+        Sleeper sleeper = Sleeper.DEFAULT;
+        switch (writeMethod) {
+          case BATCH_IMPORT:
+            throw new UnsupportedOperationException("The Batch import API is not available yet");
+          case INGEST:
+          default:
+            try {
+              if (throttler.throttleRequest(startTime)) {
+                LOG.info("Delaying request due to previous failures.");
+                this.throttledSeconds.inc(throttleWaitSeconds);
+                sleeper.sleep(throttleWaitSeconds * 1000);
+                this.throttler.successfulRequest(startTime);
+                this.successfulHL7v2MessageWrites.inc();
+              }
+              client.ingestHL7v2Message(hl7v2Store, msg.toModel());
+            } catch (Exception e) {
+              failedMessageWrites.inc();
 
 Review comment:
   all retryable errors will be retried by the request initializer in the http client.
   
   I think adding the throttling was a premature optimization on my part. We should remove adaptive throttling entirely and test this at load and see if throttling is necessary.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-604655037
 
 
   Ok an updates here from an internal thread w/ API team.
   
   1. [Message.List returning message contents is available in beta API with the view parameter.
   1. Schematized Data should be in next beta release roughly in ~2 weeks.
   1. right now the sink is outputting schematized data json wrapped in "{data=<actual_valid_json>}" 
   
   In light of these I will do the following refactors: 
   1. [x] how we batch read from to always avoid the double get. This will make it a completely parallel code path than the real-time path but I think that's ok.
   1. [ ] refactor to use beta client library (once it includes schematizedData)
       - FYI pre-work for v1beta1 migration in this branch [migration/v1beta1](https://github.com/jaketf/beam/tree/migration/v1beta1). Will merge back to this branch once we have schematizedData in beta client library.
   1. [x] I'll strip out that `{data=}` wrapper to make this easier for users.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-604655037
 
 
   Ok an updates here from an internal thread w/ API team.
   
   1. [Message.List returning message contents is available in beta API with the view parameter.
   1. Schematized Data should be in next beta release roughly in ~2 weeks.
   1. right now the sink is outputting schematized data json wrapped in "{data=<actual_valid_json>}" 
   
   In light of these I will do the following refactors: 
   1. [x] how we batch read from to always avoid the double get. This will make it a completely parallel code path than the real-time path but I think that's ok.
   1. [ ] refactor to use beta client library (once it includes schematizedData)
   1. [x] I'll strip out that `{data=}` wrapper to make this easier for users.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-603980265
 
 
   Open Questions:
   1. Should we remove adaptive throttling?
       - Seems that we're using retries in the client request initializer and right now a "bad record" will slow down the Read / Write (even though the error has nothing to do with the HL7v2 store being overwhelmed). Originally we wanted to be safe with overwhelming QPS on the HL7v2 store in batch scenarios.
   1. Should we add more to the `HealthcareIOError`?
       - Add (processing time) Timestamp? done.
       - Add a convenience DoFn `HealthcareIOErrrorToTableRowFn` to ease writing deadletter queue to BigQuery. done.
   1. Would it be more useful to expose an error rate metric than an error count?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r406476510
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   It doesn't have to be 1.30.3, it could be any later version. The purpose of them being the same is to ensure that GCP dependencies are compatible with each other so the healthcare dep doesn't break other GCP deps and vice versa.
   
   It sounds like your going to need to update google_clients_version to a later version so it supports existing GCP deps and also the new dep your adding.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] brianlucier commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
brianlucier commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r398576742
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java
 ##########
 @@ -0,0 +1,452 @@
+/*
+ * 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.client.http.HttpHeaders;
+import com.google.api.client.http.HttpRequest;
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.client.http.javanet.NetHttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.client.util.ArrayMap;
+import com.google.api.services.healthcare.v1alpha2.CloudHealthcare;
+import com.google.api.services.healthcare.v1alpha2.CloudHealthcare.Projects.Locations.Datasets.Hl7V2Stores.Messages;
+import com.google.api.services.healthcare.v1alpha2.CloudHealthcareScopes;
+import com.google.api.services.healthcare.v1alpha2.model.CreateMessageRequest;
+import com.google.api.services.healthcare.v1alpha2.model.Empty;
+import com.google.api.services.healthcare.v1alpha2.model.Hl7V2Store;
+import com.google.api.services.healthcare.v1alpha2.model.HttpBody;
+import com.google.api.services.healthcare.v1alpha2.model.IngestMessageRequest;
+import com.google.api.services.healthcare.v1alpha2.model.IngestMessageResponse;
+import com.google.api.services.healthcare.v1alpha2.model.ListMessagesResponse;
+import com.google.api.services.healthcare.v1alpha2.model.Message;
+import com.google.api.services.healthcare.v1alpha2.model.SearchResourcesRequest;
+import com.google.auth.oauth2.GoogleCredentials;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URI;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Spliterator;
+import java.util.Spliterators;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.gcp.util.RetryHttpRequestInitializer;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
+
+/**
+ * A client that talks to the Cloud Healthcare API through HTTP requests. This client is created
+ * mainly to encapsulate the unserializable dependencies, since most generated classes are not
+ * serializable in the HTTP client.
+ */
+public class HttpHealthcareApiClient<T> implements HealthcareApiClient, Serializable {
+
+  private transient CloudHealthcare client;
+
+  /**
+   * Instantiates a new Http healthcare api client.
+   *
+   * @throws IOException the io exception
+   */
+  public HttpHealthcareApiClient() throws IOException {
+    initClient();
+  }
+
+  /**
+   * Instantiates a new Http healthcare api client.
+   *
+   * @param client the client
+   * @throws IOException the io exception
+   */
+  public HttpHealthcareApiClient(CloudHealthcare client) throws IOException {
+    this.client = client;
+    initClient();
+  }
+
+  @VisibleForTesting
+  static <T, X extends Collection<T>> Stream<T> flattenIteratorCollectionsToStream(
+      Iterator<X> iterator) {
+    Spliterator<Collection<T>> spliterator = Spliterators.spliteratorUnknownSize(iterator, 0);
+    return StreamSupport.stream(spliterator, false).flatMap(Collection::stream);
+  }
+
+  public JsonFactory getJsonFactory() {
+    return this.client.getJsonFactory();
+  }
+
+  @Override
+  public ListMessagesResponse makeHL7v2ListRequest(
+      String hl7v2Store, @Nullable String filter, @Nullable String pageToken) throws IOException {
+
+    Messages.List baseRequest =
+        client
+            .projects()
+            .locations()
+            .datasets()
+            .hl7V2Stores()
+            .messages()
+            .list(hl7v2Store)
+            .setPageToken(pageToken);
+
+    if (Strings.isNullOrEmpty(filter)) {
+      return baseRequest.execute();
+    } else {
+      return baseRequest.setFilter(filter).execute();
+    }
+  }
+
+  /**
+   * Gets message id page iterator.
+   *
+   * @param hl7v2Store the HL7v2 store
+   * @return the message id page iterator
+   * @throws IOException the io exception
+   */
+  @Override
+  public Stream<String> getHL7v2MessageIDStream(String hl7v2Store) throws IOException {
+    return getHL7v2MessageIDStream(hl7v2Store, null);
+  }
+
+  /**
+   * Get a {@link Stream} of message IDs from flattening the pages of a new {@link
+   * HL7v2MessageIDPages}.
+   *
+   * @param hl7v2Store the HL7v2 store
+   * @param filter the filter
+   * @return the message id Stream
+   * @throws IOException the io exception
+   */
+  @Override
+  public Stream<String> getHL7v2MessageIDStream(String hl7v2Store, @Nullable String filter)
+      throws IOException {
+    Iterator<List<String>> iterator = new HL7v2MessageIDPages(this, hl7v2Store, filter).iterator();
+    return flattenIteratorCollectionsToStream(iterator);
+  }
+
+  /**
+   * Gets HL7v2 message.
+   *
+   * @param msgName the msg name
+   * @return the message
+   * @throws IOException the io exception
+   * @throws ParseException the parse exception
+   */
+  @Override
+  public Message getHL7v2Message(String msgName) throws IOException {
+    Message msg =
+        client.projects().locations().datasets().hl7V2Stores().messages().get(msgName).execute();
+    if (msg == null) {
+      throw new IOException(String.format("Couldn't find message: %s.", msgName));
+    }
+    return msg;
+  }
+
+  @Override
+  public Empty deleteHL7v2Message(String msgName) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .delete(msgName)
+        .execute();
+  }
+
+  /**
+   * Gets HL7v2 store.
+   *
+   * @param storeName the store name
+   * @return the HL7v2 store
+   * @throws IOException the io exception
+   */
+  @Override
+  public Hl7V2Store getHL7v2Store(String storeName) throws IOException {
+    return client.projects().locations().datasets().hl7V2Stores().get(storeName).execute();
+  }
+
+  @Override
+  public IngestMessageResponse ingestHL7v2Message(String hl7v2Store, Message msg)
+      throws IOException {
+    IngestMessageRequest ingestMessageRequest = new IngestMessageRequest();
+    ingestMessageRequest.setMessage(msg);
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .ingest(hl7v2Store, ingestMessageRequest)
+        .execute();
+  }
+
+  @Override
+  public HttpBody fhirSearch(String fhirStore, SearchResourcesRequest query) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .search(fhirStore, query)
+        .execute();
+  }
+
+  @Override
+  public Message createHL7v2Message(String hl7v2Store, Message msg) throws IOException {
+    CreateMessageRequest createMessageRequest = new CreateMessageRequest();
+    createMessageRequest.setMessage(msg);
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .create(hl7v2Store, createMessageRequest)
+        .execute();
+  }
+
+  @Override
+  public HttpBody createFhirResource(String fhirStore, String type, HttpBody body)
+      throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .create(fhirStore, type, body)
+        .execute();
+  }
+
+  @Override
+  public HttpBody executeFhirBundle(String fhirStore, HttpBody bundle) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .executeBundle(fhirStore, bundle)
+        .execute();
+  }
+
+  @Override
+  public HttpBody listFHIRResourceForPatient(String fhirStore, String patient) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .patientEverything(patient)
+        .execute();
+  }
+
+  @Override
+  public HttpBody readFHIRResource(String fhirStore, String resource) throws IOException {
+    return client.projects().locations().datasets().fhirStores().fhir().read(resource).execute();
+  }
+
+  private static class AuthenticatedRetryInitializer extends RetryHttpRequestInitializer {
+    GoogleCredentials credentials;
+
+    public AuthenticatedRetryInitializer(GoogleCredentials credentials) {
+      super();
+      this.credentials = credentials;
+    }
+
+    @Override
+    public void initialize(HttpRequest request) throws IOException {
+      super.initialize(request);
+      if (!credentials.hasRequestMetadata()) {
+        return;
+      }
+      HttpHeaders requestHeaders = request.getHeaders();
+      requestHeaders.setUserAgent("apache-beam-hl7v2-io");
+      URI uri = null;
+      if (request.getUrl() != null) {
+        uri = request.getUrl().toURI();
+      }
+      Map<String, List<String>> credentialHeaders = credentials.getRequestMetadata(uri);
+      if (credentialHeaders == null) {
+        return;
+      }
+      for (Map.Entry<String, List<String>> entry : credentialHeaders.entrySet()) {
+        String headerName = entry.getKey();
+        List<String> requestValues = new ArrayList<>(entry.getValue());
+        requestHeaders.put(headerName, requestValues);
+      }
+    }
+  }
+
+  private void initClient() throws IOException {
+    // Create a HttpRequestInitializer, which will provide a baseline configuration to all requests.
+    // HttpRequestInitializer requestInitializer = new RetryHttpRequestInitializer();
+    // GoogleCredentials credentials = GoogleCredentials.getApplicationDefault();
+    HttpRequestInitializer requestInitializer =
+        new AuthenticatedRetryInitializer(
+            GoogleCredentials.getApplicationDefault()
+                .createScoped(CloudHealthcareScopes.CLOUD_PLATFORM));
+
+    client =
+        new CloudHealthcare.Builder(new NetHttpTransport(), new GsonFactory(), requestInitializer)
+            .setApplicationName("apache-beam-hl7v2-io")
+            .build();
+  }
+
+  public static class HL7v2MessageIDPages implements Iterable<List<String>> {
+
+    private final String hl7v2Store;
+    private final String filter;
+    private transient HealthcareApiClient client;
+
+    /**
+     * Instantiates a new HL7v2 message id pages.
+     *
+     * @param client the client
+     * @param hl7v2Store the HL7v2 store
+     */
+    HL7v2MessageIDPages(HealthcareApiClient client, String hl7v2Store) {
+      this.client = client;
+      this.hl7v2Store = hl7v2Store;
+      this.filter = null;
+    }
+
+    /**
+     * Instantiates a new HL7v2 message id pages.
+     *
+     * @param client the client
+     * @param hl7v2Store the HL7v2 store
+     * @param filter the filter
+     */
+    HL7v2MessageIDPages(HealthcareApiClient client, String hl7v2Store, @Nullable String filter) {
+      this.client = client;
+      this.hl7v2Store = hl7v2Store;
+      this.filter = filter;
+    }
+
+    /**
+     * Make list request list messages response.
+     *
+     * @param client the client
+     * @param hl7v2Store the hl 7 v 2 store
+     * @param filter the filter
+     * @param pageToken the page token
+     * @return the list messages response
+     * @throws IOException the io exception
+     */
+    public static ListMessagesResponse makeListRequest(
+        HealthcareApiClient client,
+        String hl7v2Store,
+        @Nullable String filter,
+        @Nullable String pageToken)
+        throws IOException {
+      return client.makeHL7v2ListRequest(hl7v2Store, filter, pageToken);
+    }
+
+    @Override
+    public Iterator<List<String>> iterator() {
+      return new HL7v2MessageIDPagesIterator(this.client, this.hl7v2Store, this.filter);
+    }
+
+    /** The type Hl7v2 message id pages iterator. */
+    public static class HL7v2MessageIDPagesIterator implements Iterator<List<String>> {
+
+      private final String hl7v2Store;
+      private final String filter;
+      private HealthcareApiClient client;
+      private String pageToken;
+      private boolean isFirstRequest;
+
+      /**
+       * Instantiates a new Hl 7 v 2 message id pages iterator.
+       *
+       * @param client the client
+       * @param hl7v2Store the hl 7 v 2 store
+       * @param filter the filter
+       */
+      HL7v2MessageIDPagesIterator(
+          HealthcareApiClient client, String hl7v2Store, @Nullable String filter) {
+        this.client = client;
+        this.hl7v2Store = hl7v2Store;
+        this.filter = filter;
+        this.pageToken = null;
+        this.isFirstRequest = true;
+      }
+
+      @Override
+      public boolean hasNext() throws NoSuchElementException {
+        if (isFirstRequest) {
+          try {
+            ListMessagesResponse response = makeListRequest(client, hl7v2Store, filter, pageToken);
+            List<String> msgs = response.getMessages();
+            if (msgs == null) {
+              if (response.get("hl7V2Messages") != null) {
+                return ((ArrayList<ArrayMap<String, String>>) response.get("hl7V2Messages")).size()
+                    > 0;
+              }
+              return false;
+            } else {
+              return !msgs.isEmpty();
+            }
+          } catch (IOException e) {
+            throw new NoSuchElementException(
+                String.format(
+                    "Failed to list first page of HL7v2 messages from %s: %s",
+                    hl7v2Store, e.getMessage()));
+          }
+        }
+        return this.pageToken != null;
+      }
+
+      @Override
+      public List<String> next() throws NoSuchElementException {
+        try {
+          ListMessagesResponse response = makeListRequest(client, hl7v2Store, filter, pageToken);
+          this.isFirstRequest = false;
+          this.pageToken = response.getNextPageToken();
+          List<String> msgs = response.getMessages();
+          if (msgs == null && response.get("hl7V2Messages") != null) {
 
 Review comment:
   This abstract modeling of the Message would work just fine.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404367497
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,597 @@
+/*
+ * 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.v1beta1.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.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 can be fetched from the HL7v2 store in two ways Message Fetching and Message
+ * Listing.
+ *
+ * <p>Message Fetching
+ *
+ * <p>Message Fetching with {@link HL7v2IO.Read} supports use cases where you have a ${@link
+ * PCollection} of message IDS. This is appropriate for reading the HL7v2 notifications from a
+ * Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually
+ * prepared list of messages that you need to process (e.g. in a text file read with {@link
+ * org.apache.beam.sdk.io.TextIO}) .
+ *
+ * <p>Fetch Message contents from HL7v2 Store based on the {@link PCollection} of message ID strings
+ * {@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},
+ * this can be used to write to the dead letter storage system of your choosing. This error handling
+ * is mainly to catch scenarios where the upstream {@link PCollection} contains IDs that are not
+ * valid or are not reachable due to permissions issues.
+ *
+ * <p>Message Listing Message Listing with {@link HL7v2IO.ListHL7v2Messages} supports batch use
+ * cases where you want to process all the messages in an HL7v2 store or those matching a
+ * filter @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters</a>
+ * This paginates through results of a Messages.List call @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list</a>
+ * and outputs directly to a {@link PCollection} of {@link HL7v2Message}. In these use cases, the
+ * error handling similar to above is unnecessary because we are listing from the source of truth
+ * the pipeline should fail transparently if this transform fails to paginate through all the
+ * results.
+ *
+ * <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} containing the {@link HL7v2Message}
+ * 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 Read Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * PCollection<HL7v2Message> out = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2Messages(
+ *           Collections.singletonList(options.getInputHL7v2Store()), option.getHL7v2Filter()))
+ *    // Go about your happy path transformations.
+ *   .apply("Process HL7v2 Messages", ...);
+ * 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 ListHL7v2Messages} 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} message IDs which failed to be fetched from the HL7v2 store, with
+     *       error message and stacktrace.
+     * </ul>
+     */
+    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 successfulHL7v2MessageGets =
+            Metrics.counter(
+                FetchHL7v2Message.HL7v2MessageGetFn.class, "successful-hl7v2-message-gets");
+        private HealthcareApiClient client;
+
+        /** 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();
+        }
+
+        /**
+         * 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 {
+          long startTime = System.currentTimeMillis();
+
+          com.google.api.services.healthcare.v1beta1.model.Message msg =
+              client.getHL7v2Message(msgId);
+
+          if (msg == null) {
+            throw new IOException(String.format("GET request for %s returned null", msgId));
+          }
+          this.successfulHL7v2MessageGets.inc();
+          return msg;
+        }
+      }
+    }
+  }
+
+  /** List HL7v2 messages. */
+  public static class ListHL7v2Messages extends PTransform<PBegin, PCollection<HL7v2Message>> {
+    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
+     */
+    ListHL7v2Messages(List<String> hl7v2Stores, String filter) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = filter;
+    }
+
+    ListHL7v2Messages(List<String> hl7v2Stores) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = null;
+    }
+
+    @Override
+    public PCollection<HL7v2Message> expand(PBegin input) {
+      return input
+          .apply(Create.of(this.hl7v2Stores))
+          .apply(ParDo.of(new ListHL7v2MessagesFn(this.filter)));
+    }
+  }
+
+  static class ListHL7v2MessagesFn extends DoFn<String, HL7v2Message> {
+
+    private final String filter;
+    private transient HealthcareApiClient client;
+
+    /**
+     * Instantiates a new List HL7v2 fn.
+     *
+     * @param filter the filter
+     */
+    ListHL7v2MessagesFn(String filter) {
+      this.filter = filter;
+    }
+
+    /**
+     * Init client.
+     *
+     * @throws IOException the io exception
+     */
+    @Setup
+    public void initClient() throws IOException {
+      this.client = new HttpHealthcareApiClient();
+    }
+
+    /**
+     * List messages.
+     *
+     * @param context the context
+     * @throws IOException the io exception
+     */
+    @ProcessElement
+    public void listMessages(ProcessContext context) throws IOException {
+      String hl7v2Store = context.element();
+      // Output all elements of all pages.
+      this.client.getHL7v2MessageStream(hl7v2Store, this.filter).forEach(context::output);
+    }
+  }
+
+  /** The type Write. */
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<HL7v2Message>, Write.Result> {
+
+    /** The tag for the successful writes to HL7v2 store`. */
+    public static final TupleTag<HealthcareIOError<HL7v2Message>> SUCCESS =
+        new TupleTag<HealthcareIOError<HL7v2Message>>() {};
+    /** The tag for the failed writes to HL7v2 store`. */
+    public static final TupleTag<HealthcareIOError<HL7v2Message>> FAILED =
+        new TupleTag<HealthcareIOError<HL7v2Message>>() {};
+
+    /**
+     * Gets HL7v2 store.
+     *
+     * @return the HL7v2 store
+     */
+    abstract String getHL7v2Store();
+
+    /**
+     * Gets write method.
+     *
+     * @return the write method
+     */
+    abstract WriteMethod getWriteMethod();
+
+    @Override
+    public Result expand(PCollection<HL7v2Message> messages) {
+      return messages.apply(new WriteHL7v2(this.getHL7v2Store(), this.getWriteMethod()));
+    }
+
+    /** The enum Write method. */
+    public enum WriteMethod {
+      /**
+       * Ingest write method. @see <a
+       * href=https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/ingest></a>
+       */
+      INGEST,
+      /**
+       * Batch import write method. This is not yet supported by the HL7v2 API, but can be used to
+       * improve throughput once available.
+       */
+      BATCH_IMPORT
+    }
+
+    /** The type Builder. */
+    @AutoValue.Builder
+    abstract static class Builder {
+
+      /**
+       * Sets HL7v2 store.
+       *
+       * @param hl7v2Store the HL7v2 store
+       * @return the HL7v2 store
+       */
+      abstract Builder setHL7v2Store(String hl7v2Store);
+
+      /**
+       * Sets write method.
+       *
+       * @param writeMethod the write method
+       * @return the write method
+       */
+      abstract Builder setWriteMethod(WriteMethod writeMethod);
+
+      /**
+       * Build write.
+       *
+       * @return the write
+       */
+      abstract Write build();
+    }
+
+    public static class Result implements POutput {
+      private final Pipeline pipeline;
+      private final PCollection<HealthcareIOError<HL7v2Message>> failedInsertsWithErr;
+
+      /** Creates a {@link HL7v2IO.Write.Result} in the given {@link Pipeline}. */
+      static Result in(
+          Pipeline pipeline, PCollection<HealthcareIOError<HL7v2Message>> failedInserts) {
+        return new Result(pipeline, failedInserts);
+      }
+
+      public PCollection<HealthcareIOError<HL7v2Message>> getFailedInsertsWithErr() {
+        return this.failedInsertsWithErr;
+      }
+
+      @Override
+      public Pipeline getPipeline() {
+        return this.pipeline;
+      }
+
+      @Override
+      public Map<TupleTag<?>, PValue> expand() {
+        failedInsertsWithErr.setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder()));
+        return ImmutableMap.of(FAILED, failedInsertsWithErr);
+      }
+
+      @Override
+      public void finishSpecifyingOutput(
+          String transformName, PInput input, PTransform<?, ?> transform) {}
+
+      private Result(
+          Pipeline pipeline, PCollection<HealthcareIOError<HL7v2Message>> failedInsertsWithErr) {
+        this.pipeline = pipeline;
+        this.failedInsertsWithErr = failedInsertsWithErr;
+      }
+    }
+  }
+
+  /** The type Write hl 7 v 2. */
+  static class WriteHL7v2 extends PTransform<PCollection<HL7v2Message>, Write.Result> {
+    private final String hl7v2Store;
+    private final Write.WriteMethod writeMethod;
+
+    /**
+     * Instantiates a new Write hl 7 v 2.
+     *
+     * @param hl7v2Store the hl 7 v 2 store
+     * @param writeMethod the write method
+     */
+    WriteHL7v2(String hl7v2Store, Write.WriteMethod writeMethod) {
+      this.hl7v2Store = hl7v2Store;
+      this.writeMethod = writeMethod;
+    }
+
+    @Override
+    public Write.Result expand(PCollection<HL7v2Message> input) {
+      PCollection<HealthcareIOError<HL7v2Message>> failedInserts =
+          input
+              .apply(ParDo.of(new WriteHL7v2Fn(hl7v2Store, writeMethod)))
+              .setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder()));
+      return Write.Result.in(input.getPipeline(), failedInserts);
+    }
+
+    /** The type Write hl 7 v 2 fn. */
+    static class WriteHL7v2Fn extends DoFn<HL7v2Message, HealthcareIOError<HL7v2Message>> {
+      // TODO when the healthcare API releases a bulk import method this should use that to improve
+      // throughput.
+
+      private Counter failedMessageWrites =
+          Metrics.counter(WriteHL7v2Fn.class, "failed-hl7v2-message-writes");
+      private final String hl7v2Store;
+      private final Counter successfulHL7v2MessageWrites =
+          Metrics.counter(WriteHL7v2.class, "successful-hl7v2-message-writes");
+      private final Write.WriteMethod writeMethod;
+
+      private static final Logger LOG = LoggerFactory.getLogger(WriteHL7v2.WriteHL7v2Fn.class);
+      private transient HealthcareApiClient client;
+
+      /**
+       * Instantiates a new Write HL7v2 fn.
+       *
+       * @param hl7v2Store the HL7v2 store
+       * @param writeMethod the write method
+       */
+      WriteHL7v2Fn(String hl7v2Store, Write.WriteMethod writeMethod) {
+        this.hl7v2Store = hl7v2Store;
+        this.writeMethod = writeMethod;
+      }
+
+      /**
+       * Init client.
+       *
+       * @throws IOException the io exception
+       */
+      @Setup
+      public void initClient() throws IOException {
+        this.client = new HttpHealthcareApiClient();
+      }
+
+      /**
+       * Write messages.
+       *
+       * @param context the context
+       */
+      @ProcessElement
+      public void writeMessages(ProcessContext context) {
+        HL7v2Message msg = context.element();
+        long startTime = System.currentTimeMillis();
+        Sleeper sleeper = Sleeper.DEFAULT;
+        switch (writeMethod) {
+          case BATCH_IMPORT:
+            throw new UnsupportedOperationException("The Batch import API is not available yet");
+          case INGEST:
+          default:
+            try {
+              client.ingestHL7v2Message(hl7v2Store, msg.toModel());
 
 Review comment:
   The service does not support batching *yet*.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-611091323
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-611610963
 
 
   > > 3. What is the feasibility / process for back-porting this to prior beam releases for expediting use by users who want to use an already released version e.g. 2.19 ?
   > 
   > bumping this question. would like to know timing on when I can see it in a released version... either backport, or a new one.
   
   You would need to get support from the community to do the backport and release which seems unlikely based upon this being a new feature (vs a critical bug/security issue) and people are constrained time wise due to COVID-19.
   
   Once this is merged it would be part of the subsequent release, the branch cut dates are here:  https://calendar.google.com/calendar/embed?src=0p73sl034k80oob7seouanigd0%40group.calendar.google.com (with 2.22.0 scheduled on May 20th)

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612259919
 
 
   @lukecwik would it be a reasonable compromise to introduce a `google_oauth_clients_verion`? If not I believe we are at an impass.
   Reasoning / Research / Background:
   - features in healthcare API making this PR possible require >=1.30.9 (healthcare api absolute minimum version is 1.30.8)
   - all deps under `com.google.apis` are updatable to 1.30.9
   - the [two](https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client/) [deps](https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client-java6/) under `com.google.oauth-client` have versions that stop at 1.30.6
   - I have a suspicion that this 1.30.9 for `com.google.apis` is not even really related to the 1.30.6 for `com.google.oauth-clients` but an unfortunate naming convention similarity
       - see that 1.30.6 [oauth client updated fairly recently on 2020-03-05](https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client/1.30.6/) and is required by the version of [bigdata.oss:gcsio 2.1.2](https://mvnrepository.com/artifact/com.google.cloud.bigdataoss/gcsio/2.1.2) we just updated to. 
   
   I've prepared this in the latest commit 
   59dc3c2  to make this a concrete proposal to review / run pre/post commits on.
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613112821
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614374497
 
 
   And liftoff

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614204847
 
 
   @suztomo - Tomo are you able to review the dependency changes for this PR?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404368939
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,597 @@
+/*
+ * 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.v1beta1.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.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 can be fetched from the HL7v2 store in two ways Message Fetching and Message
+ * Listing.
+ *
+ * <p>Message Fetching
+ *
+ * <p>Message Fetching with {@link HL7v2IO.Read} supports use cases where you have a ${@link
+ * PCollection} of message IDS. This is appropriate for reading the HL7v2 notifications from a
+ * Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually
+ * prepared list of messages that you need to process (e.g. in a text file read with {@link
+ * org.apache.beam.sdk.io.TextIO}) .
+ *
+ * <p>Fetch Message contents from HL7v2 Store based on the {@link PCollection} of message ID strings
+ * {@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},
+ * this can be used to write to the dead letter storage system of your choosing. This error handling
+ * is mainly to catch scenarios where the upstream {@link PCollection} contains IDs that are not
+ * valid or are not reachable due to permissions issues.
+ *
+ * <p>Message Listing Message Listing with {@link HL7v2IO.ListHL7v2Messages} supports batch use
+ * cases where you want to process all the messages in an HL7v2 store or those matching a
+ * filter @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters</a>
+ * This paginates through results of a Messages.List call @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list</a>
+ * and outputs directly to a {@link PCollection} of {@link HL7v2Message}. In these use cases, the
+ * error handling similar to above is unnecessary because we are listing from the source of truth
+ * the pipeline should fail transparently if this transform fails to paginate through all the
+ * results.
+ *
+ * <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} containing the {@link HL7v2Message}
+ * 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 Read Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * PCollection<HL7v2Message> out = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2Messages(
+ *           Collections.singletonList(options.getInputHL7v2Store()), option.getHL7v2Filter()))
+ *    // Go about your happy path transformations.
+ *   .apply("Process HL7v2 Messages", ...);
+ * 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) {
 
 Review comment:
   This is supposed to mirror the [ingest REST API method](https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/ingest) that's used. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612234407
 
 
   > @lukecwik I ran the linkeage check "as-is" ([log](https://gist.github.com/suztomo/3178f298bd315e2cc3897b5528a961b8)) and IIUC this has no issues.
   > 
   > Is this acceptable or is it a hard requirement that I update google_clients_version?
   
   This is a hard requirement. Either we are using the google_clients_version of shared deps or google_healthcare_clients_version for deps. We can't be using both.
   
   You'll want to have a separate PR that just updates the google_clients version to the version that you need for the healthcare lib.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614201101
 
 
   > @lukecwik please let me know if there's anything else I can do to give confidence in the dependency changes.
   
   I have a lot of reviews on my radar and will get to this one as I can. If you want, you can reach out to the community for another reviewer to double check the dependency work and/or have Pablo take it up.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-600786892
 
 
   ("retest this please" is a code phrase for jenkins to run precommits on the PR)

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614249032
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612242130
 
 
   > @lukecwik that's what I thought to but when I do that I get the output in [this comment](https://github.com/apache/beam/pull/11151#discussion_r406551322)
   
   A lot of google libraries have three parts to the version string (API major version, API minor version, shared lib version)
   
   You have to find and upgrade the client libraries to versions that contain the 1.30.9 shared lib version like:
   google-api-services-cloudresourcemanager:v1-rev20191206-1.30.4 becomes google-api-services-cloudresourcemanager:v1-rev20200311-1.30.9
   
   Best way to find matching versions is to look at the index of the maven repo for each artifact (e.g.):
   https://repo1.maven.org/maven2/com/google/apis/google-api-services-cloudresourcemanager/
   
   Find ones that have the same API major version and any newer API minor version with the shard lib version being 1.30.9.
   
   Major version changes are breaking API changes, minor version increases are meant to be safe to upgrade and shared lib version just needs to be the same across all versions.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404358283
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,636 @@
+/*
+ * 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
 
 Review comment:
   How would one build a transform that reads the whole HL8v2Store (as opposed to individual message IDs)?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-610050634
 
 
   Next Steps (based on offline feedback):
   - [x] Improve API for users:
       - [x] Add static methods for common patterns with `ListHL7v2Messages` 
       - [x] Add `ValueProvider<String>` support to ease use in the DataflowTemplates
           - [x] `ListHL7v2Messages` (hl7v2Store and filter)
           - [x] `Write` (hl7v2Store)
   - [ ] "standardize" integration tests
       - [x] Refactor ITs to create / destroy HL7v2 Store under a parameterized dataset in `@BeforeClass` `@AfterClass` to avoid issues with parallel tests runs.
       - [x] Remove hard coding of my HL7v2Store / project in integration tests.
       - [x] Add Healthcare API Dataset to Beam integration test project (pending permissions in [this dev list thread](https://lists.apache.org/thread.html/rebe5cd40a40a9fc7f2c1d563b48ee1ce4ff9cac3dfdc0258006cc686%40%3Cdev.beam.apache.org%3E))

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] brianlucier commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
brianlucier commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r398576097
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java
 ##########
 @@ -0,0 +1,452 @@
+/*
+ * 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.client.http.HttpHeaders;
+import com.google.api.client.http.HttpRequest;
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.client.http.javanet.NetHttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.client.util.ArrayMap;
+import com.google.api.services.healthcare.v1alpha2.CloudHealthcare;
+import com.google.api.services.healthcare.v1alpha2.CloudHealthcare.Projects.Locations.Datasets.Hl7V2Stores.Messages;
+import com.google.api.services.healthcare.v1alpha2.CloudHealthcareScopes;
+import com.google.api.services.healthcare.v1alpha2.model.CreateMessageRequest;
+import com.google.api.services.healthcare.v1alpha2.model.Empty;
+import com.google.api.services.healthcare.v1alpha2.model.Hl7V2Store;
+import com.google.api.services.healthcare.v1alpha2.model.HttpBody;
+import com.google.api.services.healthcare.v1alpha2.model.IngestMessageRequest;
+import com.google.api.services.healthcare.v1alpha2.model.IngestMessageResponse;
+import com.google.api.services.healthcare.v1alpha2.model.ListMessagesResponse;
+import com.google.api.services.healthcare.v1alpha2.model.Message;
+import com.google.api.services.healthcare.v1alpha2.model.SearchResourcesRequest;
+import com.google.auth.oauth2.GoogleCredentials;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URI;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Spliterator;
+import java.util.Spliterators;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.gcp.util.RetryHttpRequestInitializer;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
+
+/**
+ * A client that talks to the Cloud Healthcare API through HTTP requests. This client is created
+ * mainly to encapsulate the unserializable dependencies, since most generated classes are not
+ * serializable in the HTTP client.
+ */
+public class HttpHealthcareApiClient<T> implements HealthcareApiClient, Serializable {
+
+  private transient CloudHealthcare client;
+
+  /**
+   * Instantiates a new Http healthcare api client.
+   *
+   * @throws IOException the io exception
+   */
+  public HttpHealthcareApiClient() throws IOException {
+    initClient();
+  }
+
+  /**
+   * Instantiates a new Http healthcare api client.
+   *
+   * @param client the client
+   * @throws IOException the io exception
+   */
+  public HttpHealthcareApiClient(CloudHealthcare client) throws IOException {
+    this.client = client;
+    initClient();
+  }
+
+  @VisibleForTesting
+  static <T, X extends Collection<T>> Stream<T> flattenIteratorCollectionsToStream(
+      Iterator<X> iterator) {
+    Spliterator<Collection<T>> spliterator = Spliterators.spliteratorUnknownSize(iterator, 0);
+    return StreamSupport.stream(spliterator, false).flatMap(Collection::stream);
+  }
+
+  public JsonFactory getJsonFactory() {
+    return this.client.getJsonFactory();
+  }
+
+  @Override
+  public ListMessagesResponse makeHL7v2ListRequest(
+      String hl7v2Store, @Nullable String filter, @Nullable String pageToken) throws IOException {
+
+    Messages.List baseRequest =
+        client
+            .projects()
+            .locations()
+            .datasets()
+            .hl7V2Stores()
+            .messages()
+            .list(hl7v2Store)
+            .setPageToken(pageToken);
+
+    if (Strings.isNullOrEmpty(filter)) {
+      return baseRequest.execute();
+    } else {
+      return baseRequest.setFilter(filter).execute();
+    }
+  }
+
+  /**
+   * Gets message id page iterator.
+   *
+   * @param hl7v2Store the HL7v2 store
+   * @return the message id page iterator
+   * @throws IOException the io exception
+   */
+  @Override
+  public Stream<String> getHL7v2MessageIDStream(String hl7v2Store) throws IOException {
+    return getHL7v2MessageIDStream(hl7v2Store, null);
+  }
+
+  /**
+   * Get a {@link Stream} of message IDs from flattening the pages of a new {@link
+   * HL7v2MessageIDPages}.
+   *
+   * @param hl7v2Store the HL7v2 store
+   * @param filter the filter
+   * @return the message id Stream
+   * @throws IOException the io exception
+   */
+  @Override
+  public Stream<String> getHL7v2MessageIDStream(String hl7v2Store, @Nullable String filter)
+      throws IOException {
+    Iterator<List<String>> iterator = new HL7v2MessageIDPages(this, hl7v2Store, filter).iterator();
+    return flattenIteratorCollectionsToStream(iterator);
+  }
+
+  /**
+   * Gets HL7v2 message.
+   *
+   * @param msgName the msg name
+   * @return the message
+   * @throws IOException the io exception
+   * @throws ParseException the parse exception
+   */
+  @Override
+  public Message getHL7v2Message(String msgName) throws IOException {
+    Message msg =
+        client.projects().locations().datasets().hl7V2Stores().messages().get(msgName).execute();
+    if (msg == null) {
+      throw new IOException(String.format("Couldn't find message: %s.", msgName));
+    }
+    return msg;
+  }
+
+  @Override
+  public Empty deleteHL7v2Message(String msgName) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .delete(msgName)
+        .execute();
+  }
+
+  /**
+   * Gets HL7v2 store.
+   *
+   * @param storeName the store name
+   * @return the HL7v2 store
+   * @throws IOException the io exception
+   */
+  @Override
+  public Hl7V2Store getHL7v2Store(String storeName) throws IOException {
+    return client.projects().locations().datasets().hl7V2Stores().get(storeName).execute();
+  }
+
+  @Override
+  public IngestMessageResponse ingestHL7v2Message(String hl7v2Store, Message msg)
+      throws IOException {
+    IngestMessageRequest ingestMessageRequest = new IngestMessageRequest();
+    ingestMessageRequest.setMessage(msg);
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .ingest(hl7v2Store, ingestMessageRequest)
+        .execute();
+  }
 
 Review comment:
   Does the client handle quota-based retries on 429 responses?  Or is left to the caller to handle retries?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-610050634
 
 
   Next Steps (based on offline feed):
   - [x] Improve API for users:
       - [x] Add static methods for common patterns with `ListHL7v2Messages` 
       - [x] Add `ValueProvider<String>` support to ease use in the DataflowTemplates
           - [x] `ListHL7v2Messages` (hl7v2Store and filter)
           - [x] `Write` (hl7v2Store)
   - [ ] "standardize" integration tests
       - [x] Refactor ITs to create / destroy HL7v2 Store under a parameterized dataset in `@BeforeClass` `@AfterClass` to avoid issues with parallel tests runs.
       - [x] Remove hard coding of my HL7v2Store / project in integration tests.
       - [ ] Add Healthcare API Dataset to Beam integration test project

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614337568
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612275337
 
 
   Run Java PreCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612245275
 
 
   @lukecwik 
   some progress I was able to update storage / cloudresourcemanager that way.
   
   However, in
   https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client/
   https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client-java6/
   
   They stop at 1.30.6
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r395269212
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,620 @@
+/*
+ * 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.IngestMessageResponse;
+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.Collections;
+import java.util.List;
+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.PDone;
+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.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 HL7v2 Messages are fetched from the HL7v2 store based on the {@link PCollection} of of
+ * message IDs {@link String}s produced by the {@link AutoValue_HL7v2IO_Read#getMessageIDTransform}
+ * as {@link PCollectionTuple}*** containing an {@link HL7v2IO.Read#OUT} tag for successfully
 
 Review comment:
   Agreed. This is cleaner, I never thought of extending `POutput`, thanks for the tip!

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r398949505
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,636 @@
+/*
+ * 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
 
 Review comment:
   @brianlucier PTAL at this updated doc string. 
   it describes my latest change in ba9d023 to avoid the double get whenever we reading a whole HL7v2Store with Messages.List API by adding the view=FULL param.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613058131
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-610050634
 
 
   Next Steps (based on offline feedback):
   - [x] Improve API for users:
       - [x] Add static methods for common patterns with `ListHL7v2Messages` 
       - [x] Add `ValueProvider<String>` support to ease use in the DataflowTemplates
           - [x] `ListHL7v2Messages` (hl7v2Store and filter)
           - [x] `Write` (hl7v2Store)
   - [ ] "standardize" integration tests
       - [x] Refactor ITs to create / destroy HL7v2 Store under a parameterized dataset in `@BeforeClass` `@AfterClass` to avoid issues with parallel tests runs.
       - [x] Remove hard coding of my HL7v2Store / project in integration tests.
       - [ ] Add Healthcare API Dataset to Beam integration test project

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-610707474
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-604001261
 
 
   Future improvements:
   - Currently this uses the alpha API because the motivating use case of HL7v2 -> FHIR mapping which requires the `schematizedData` field (not yet available in beta API).
   - For backwards compatibility the `ListHL7v2MessageIDs` always uses Messages.List to get message IDs and relies on the the `HL7v2IO.Read` to fetch the message contents. This provides a consistent flow for real-time and batch and provides backwards compatibility. However, in the alpha API Messages.List returns the actual message contents (rather than just the message IDs). This leads to us reading the messages contents twice for alpha HL7v2 stores. I suggest we address optimizing this "double fetch" in a future PR as the alpha API stabilizes.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612191775
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614206162
 
 
   @suztomo would you be able take a look at the dependency changes? otherwise I will drop a note to the dev list.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-610003502
 
 
   in case it's not obvious: feel free to ignore failures in unrelated test suites (e.g. communitymetrics)

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r407567891
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   It turns out that someone did the work to upgrade google_clients_version to 1.30.9 in #11208 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614313665
 
 
   @pabloem reshuffle added and ITs passing locally as of c50df5f

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-604655037
 
 
   Ok an updates here from an internal thread w/ API team.
   
   1. Message.List returning message contents is available in beta API with the view parameter.
   1. Schematized Data should be in next beta release roughly in ~2 weeks.
   1. right now the sink is outputting schematized data json wrapped in "{data=<actual_valid_json>}" 
   
   In light of these I will do the following refactors: 
   1. how we batch read from to always avoid the double get. This will make it a completely parallel code path than the real-time path but I think that's ok.
   1. refactor to use beta client library (once it includes schematizedData)
   1. I'll strip out that `{data=}` wrapper to make this easier for users.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612269800
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r394631597
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,620 @@
+/*
+ * 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.IngestMessageResponse;
+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.Collections;
+import java.util.List;
+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.PDone;
+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.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 HL7v2 Messages are fetched from the HL7v2 store based on the {@link PCollection} of of
+ * message IDs {@link String}s produced by the {@link AutoValue_HL7v2IO_Read#getMessageIDTransform}
+ * as {@link PCollectionTuple}*** containing an {@link HL7v2IO.Read#OUT} tag for successfully
 
 Review comment:
   I would recommend that the output of the transform be something like [WriteResult](https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java#L34) returned by [BigQueryIO.Write](https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java#L1720).
   
   That should give you a more digestable output type for new users, with things like `getFailedElements`, etc. Thoughts?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404356635
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,597 @@
+/*
+ * 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.v1beta1.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.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 can be fetched from the HL7v2 store in two ways Message Fetching and Message
+ * Listing.
+ *
+ * <p>Message Fetching
+ *
+ * <p>Message Fetching with {@link HL7v2IO.Read} supports use cases where you have a ${@link
+ * PCollection} of message IDS. This is appropriate for reading the HL7v2 notifications from a
+ * Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually
+ * prepared list of messages that you need to process (e.g. in a text file read with {@link
+ * org.apache.beam.sdk.io.TextIO}) .
+ *
+ * <p>Fetch Message contents from HL7v2 Store based on the {@link PCollection} of message ID strings
+ * {@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},
+ * this can be used to write to the dead letter storage system of your choosing. This error handling
+ * is mainly to catch scenarios where the upstream {@link PCollection} contains IDs that are not
+ * valid or are not reachable due to permissions issues.
+ *
+ * <p>Message Listing Message Listing with {@link HL7v2IO.ListHL7v2Messages} supports batch use
+ * cases where you want to process all the messages in an HL7v2 store or those matching a
+ * filter @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters</a>
+ * This paginates through results of a Messages.List call @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list</a>
+ * and outputs directly to a {@link PCollection} of {@link HL7v2Message}. In these use cases, the
+ * error handling similar to above is unnecessary because we are listing from the source of truth
+ * the pipeline should fail transparently if this transform fails to paginate through all the
+ * results.
+ *
+ * <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} containing the {@link HL7v2Message}
+ * 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 Read Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * PCollection<HL7v2Message> out = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2Messages(
+ *           Collections.singletonList(options.getInputHL7v2Store()), option.getHL7v2Filter()))
+ *    // Go about your happy path transformations.
+ *   .apply("Process HL7v2 Messages", ...);
+ * 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 ListHL7v2Messages} 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} message IDs which failed to be fetched from the HL7v2 store, with
+     *       error message and stacktrace.
+     * </ul>
+     */
+    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 successfulHL7v2MessageGets =
+            Metrics.counter(
+                FetchHL7v2Message.HL7v2MessageGetFn.class, "successful-hl7v2-message-gets");
+        private HealthcareApiClient client;
+
+        /** 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();
+        }
+
+        /**
+         * 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 {
+          long startTime = System.currentTimeMillis();
+
+          com.google.api.services.healthcare.v1beta1.model.Message msg =
 
 Review comment:
   This is the same as the Write transform. It may be useful to batch multiple fetches in a single request if possible. This should be fine for now, as workers / PubSub are able to scale to very high parallelism in streaming, but may be a valuable optimization later. (or now if you have the time / the HL7v2 service supports it)

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] brianlucier commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
brianlucier commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r398584626
 
 

 ##########
 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:
   As a user -- I'll know what version of the V2 store I am using and likely am never switching back and forth.  For instance, I would just ignore the beta PTransform and use the alpha one.  Seems like that is not a large user burden.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613113280
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612232240
 
 
   @lukecwik I ran the linkeage check "as-is" ([log](https://gist.github.com/suztomo/3178f298bd315e2cc3897b5528a961b8)) and IIUC this has no issues.
   
   Is this acceptable or is it a hard requirement that I update google_clients_version?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614239497
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612232240
 
 
   @lukecwik I ran the linkeage check "as-is" ([log](https://gist.github.com/suztomo/3178f298bd315e2cc3897b5528a961b8)) and IIUC this has no issues.
   
   Is this acceptable or is it a hard requirement that I update google_clients_version?
   
   pablo recommended we add a: 
   ```groovy
   // TODO(BEAM-XXXX) when google_clients_version is updated >=1.30.9 use that instead of google_cloud_healthcare_version
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r406551322
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   I'm rather new to these generated clients and beam.
   Is there any process this community has followed in the past for finding a suitable version in situations like this?
   Seems like a lot for this PR to make this rather global upgrade.
   Naturally if I just flip google_clients_version to 1.30.9 it breaks  a lot:
   ```
   Could not determine the dependencies of task ':sdks:java:harness:shadowJar'.
   > Could not resolve all dependencies for configuration ':sdks:java:harness:runtimeClasspath'.
      > Could not find com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-1.30.9.
        Required by:
            project :sdks:java:harness > project :sdks:java:extensions:google-cloud-platform-core
      > Could not find com.google.apis:google-api-services-storage:v1-rev20191011-1.30.9.
        Required by:
            project :sdks:java:harness > project :sdks:java:extensions:google-cloud-platform-core
      > Could not find com.google.oauth-client:google-oauth-client:1.30.9.
      > Could not find com.google.apis:google-api-services-storage:v1-rev20191011-1.30.9.
        Required by:
            project :sdks:java:harness > project :sdks:java:extensions:google-cloud-platform-core > com.google.cloud.bigdataoss:gcsio:2.0.0
            project :sdks:java:harness > project :sdks:java:extensions:google-cloud-platform-core > com.google.cloud.bigdataoss:util:2.0.0
      > Could not find com.google.oauth-client:google-oauth-client:1.30.9.
        Required by:
            project :sdks:java:harness > project :sdks:java:extensions:google-cloud-platform-core > com.google.cloud.bigdataoss:gcsio:2.0.0
            project :sdks:java:harness > project :sdks:java:extensions:google-cloud-platform-core > com.google.cloud.bigdataoss:util:2.0.0
      > Could not find com.google.oauth-client:google-oauth-client-java6:1.30.9.
        Required by:
            project :sdks:java:harness > project :sdks:java:extensions:google-cloud-platform-core > com.google.cloud.bigdataoss:gcsio:2.0.0
            project :sdks:java:harness > project :sdks:java:extensions:google-cloud-platform-core > com.google.cloud.bigdataoss:util:2.0.0
      > Could not find com.google.oauth-client:google-oauth-client-java6:1.30.9.
        Required by:
            project :sdks:java:harness > project :sdks:java:extensions:google-cloud-platform-core > com.google.cloud.bigdataoss:gcsio:2.0.0 > com.google.api-client:google-api-client-java6:1.30.9
   * 
   ```
   
   seems like our culprits are 
   - [ ] `google-oauth-client`
   - [ ] `google-oauth-client-java6`
   - [ ] `google-api-services-storage`
   
   If I go in and try to massage the revision numbers to make this work how will I ensure that all those reliant on those deps are still working? Is that covered in pre/post commit ITs?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613120178
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614325022
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404347581
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,597 @@
+/*
+ * 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.v1beta1.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.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 can be fetched from the HL7v2 store in two ways Message Fetching and Message
+ * Listing.
+ *
+ * <p>Message Fetching
+ *
+ * <p>Message Fetching with {@link HL7v2IO.Read} supports use cases where you have a ${@link
+ * PCollection} of message IDS. This is appropriate for reading the HL7v2 notifications from a
+ * Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually
+ * prepared list of messages that you need to process (e.g. in a text file read with {@link
+ * org.apache.beam.sdk.io.TextIO}) .
+ *
+ * <p>Fetch Message contents from HL7v2 Store based on the {@link PCollection} of message ID strings
+ * {@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},
+ * this can be used to write to the dead letter storage system of your choosing. This error handling
+ * is mainly to catch scenarios where the upstream {@link PCollection} contains IDs that are not
+ * valid or are not reachable due to permissions issues.
+ *
+ * <p>Message Listing Message Listing with {@link HL7v2IO.ListHL7v2Messages} supports batch use
+ * cases where you want to process all the messages in an HL7v2 store or those matching a
+ * filter @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters</a>
+ * This paginates through results of a Messages.List call @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list</a>
+ * and outputs directly to a {@link PCollection} of {@link HL7v2Message}. In these use cases, the
+ * error handling similar to above is unnecessary because we are listing from the source of truth
+ * the pipeline should fail transparently if this transform fails to paginate through all the
+ * results.
+ *
+ * <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} containing the {@link HL7v2Message}
+ * 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 Read Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * PCollection<HL7v2Message> out = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2Messages(
+ *           Collections.singletonList(options.getInputHL7v2Store()), option.getHL7v2Filter()))
+ *    // Go about your happy path transformations.
+ *   .apply("Process HL7v2 Messages", ...);
+ * 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 ListHL7v2Messages} 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} message IDs which failed to be fetched from the HL7v2 store, with
+     *       error message and stacktrace.
+     * </ul>
+     */
+    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 successfulHL7v2MessageGets =
+            Metrics.counter(
+                FetchHL7v2Message.HL7v2MessageGetFn.class, "successful-hl7v2-message-gets");
+        private HealthcareApiClient client;
+
+        /** 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();
+        }
+
+        /**
+         * 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 {
+          long startTime = System.currentTimeMillis();
+
+          com.google.api.services.healthcare.v1beta1.model.Message msg =
+              client.getHL7v2Message(msgId);
+
+          if (msg == null) {
+            throw new IOException(String.format("GET request for %s returned null", msgId));
+          }
+          this.successfulHL7v2MessageGets.inc();
+          return msg;
+        }
+      }
+    }
+  }
+
+  /** List HL7v2 messages. */
+  public static class ListHL7v2Messages extends PTransform<PBegin, PCollection<HL7v2Message>> {
+    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
+     */
+    ListHL7v2Messages(List<String> hl7v2Stores, String filter) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = filter;
+    }
+
+    ListHL7v2Messages(List<String> hl7v2Stores) {
+      this.hl7v2Stores = hl7v2Stores;
+      this.filter = null;
+    }
+
+    @Override
+    public PCollection<HL7v2Message> expand(PBegin input) {
+      return input
+          .apply(Create.of(this.hl7v2Stores))
+          .apply(ParDo.of(new ListHL7v2MessagesFn(this.filter)));
+    }
+  }
+
+  static class ListHL7v2MessagesFn extends DoFn<String, HL7v2Message> {
+
+    private final String filter;
+    private transient HealthcareApiClient client;
+
+    /**
+     * Instantiates a new List HL7v2 fn.
+     *
+     * @param filter the filter
+     */
+    ListHL7v2MessagesFn(String filter) {
+      this.filter = filter;
+    }
+
+    /**
+     * Init client.
+     *
+     * @throws IOException the io exception
+     */
+    @Setup
+    public void initClient() throws IOException {
+      this.client = new HttpHealthcareApiClient();
+    }
+
+    /**
+     * List messages.
+     *
+     * @param context the context
+     * @throws IOException the io exception
+     */
+    @ProcessElement
+    public void listMessages(ProcessContext context) throws IOException {
+      String hl7v2Store = context.element();
+      // Output all elements of all pages.
+      this.client.getHL7v2MessageStream(hl7v2Store, this.filter).forEach(context::output);
+    }
+  }
+
+  /** The type Write. */
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<HL7v2Message>, Write.Result> {
+
+    /** The tag for the successful writes to HL7v2 store`. */
+    public static final TupleTag<HealthcareIOError<HL7v2Message>> SUCCESS =
+        new TupleTag<HealthcareIOError<HL7v2Message>>() {};
+    /** The tag for the failed writes to HL7v2 store`. */
+    public static final TupleTag<HealthcareIOError<HL7v2Message>> FAILED =
+        new TupleTag<HealthcareIOError<HL7v2Message>>() {};
+
+    /**
+     * Gets HL7v2 store.
+     *
+     * @return the HL7v2 store
+     */
+    abstract String getHL7v2Store();
+
+    /**
+     * Gets write method.
+     *
+     * @return the write method
+     */
+    abstract WriteMethod getWriteMethod();
+
+    @Override
+    public Result expand(PCollection<HL7v2Message> messages) {
+      return messages.apply(new WriteHL7v2(this.getHL7v2Store(), this.getWriteMethod()));
+    }
+
+    /** The enum Write method. */
+    public enum WriteMethod {
+      /**
+       * Ingest write method. @see <a
+       * href=https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/ingest></a>
+       */
+      INGEST,
+      /**
+       * Batch import write method. This is not yet supported by the HL7v2 API, but can be used to
+       * improve throughput once available.
+       */
+      BATCH_IMPORT
+    }
+
+    /** The type Builder. */
+    @AutoValue.Builder
+    abstract static class Builder {
+
+      /**
+       * Sets HL7v2 store.
+       *
+       * @param hl7v2Store the HL7v2 store
+       * @return the HL7v2 store
+       */
+      abstract Builder setHL7v2Store(String hl7v2Store);
+
+      /**
+       * Sets write method.
+       *
+       * @param writeMethod the write method
+       * @return the write method
+       */
+      abstract Builder setWriteMethod(WriteMethod writeMethod);
+
+      /**
+       * Build write.
+       *
+       * @return the write
+       */
+      abstract Write build();
+    }
+
+    public static class Result implements POutput {
+      private final Pipeline pipeline;
+      private final PCollection<HealthcareIOError<HL7v2Message>> failedInsertsWithErr;
+
+      /** Creates a {@link HL7v2IO.Write.Result} in the given {@link Pipeline}. */
+      static Result in(
+          Pipeline pipeline, PCollection<HealthcareIOError<HL7v2Message>> failedInserts) {
+        return new Result(pipeline, failedInserts);
+      }
+
+      public PCollection<HealthcareIOError<HL7v2Message>> getFailedInsertsWithErr() {
+        return this.failedInsertsWithErr;
+      }
+
+      @Override
+      public Pipeline getPipeline() {
+        return this.pipeline;
+      }
+
+      @Override
+      public Map<TupleTag<?>, PValue> expand() {
+        failedInsertsWithErr.setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder()));
+        return ImmutableMap.of(FAILED, failedInsertsWithErr);
+      }
+
+      @Override
+      public void finishSpecifyingOutput(
+          String transformName, PInput input, PTransform<?, ?> transform) {}
+
+      private Result(
+          Pipeline pipeline, PCollection<HealthcareIOError<HL7v2Message>> failedInsertsWithErr) {
+        this.pipeline = pipeline;
+        this.failedInsertsWithErr = failedInsertsWithErr;
+      }
+    }
+  }
+
+  /** The type Write hl 7 v 2. */
+  static class WriteHL7v2 extends PTransform<PCollection<HL7v2Message>, Write.Result> {
+    private final String hl7v2Store;
+    private final Write.WriteMethod writeMethod;
+
+    /**
+     * Instantiates a new Write hl 7 v 2.
+     *
+     * @param hl7v2Store the hl 7 v 2 store
+     * @param writeMethod the write method
+     */
+    WriteHL7v2(String hl7v2Store, Write.WriteMethod writeMethod) {
+      this.hl7v2Store = hl7v2Store;
+      this.writeMethod = writeMethod;
+    }
+
+    @Override
+    public Write.Result expand(PCollection<HL7v2Message> input) {
+      PCollection<HealthcareIOError<HL7v2Message>> failedInserts =
+          input
+              .apply(ParDo.of(new WriteHL7v2Fn(hl7v2Store, writeMethod)))
+              .setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder()));
+      return Write.Result.in(input.getPipeline(), failedInserts);
+    }
+
+    /** The type Write hl 7 v 2 fn. */
+    static class WriteHL7v2Fn extends DoFn<HL7v2Message, HealthcareIOError<HL7v2Message>> {
+      // TODO when the healthcare API releases a bulk import method this should use that to improve
+      // throughput.
+
+      private Counter failedMessageWrites =
+          Metrics.counter(WriteHL7v2Fn.class, "failed-hl7v2-message-writes");
+      private final String hl7v2Store;
+      private final Counter successfulHL7v2MessageWrites =
+          Metrics.counter(WriteHL7v2.class, "successful-hl7v2-message-writes");
+      private final Write.WriteMethod writeMethod;
+
+      private static final Logger LOG = LoggerFactory.getLogger(WriteHL7v2.WriteHL7v2Fn.class);
+      private transient HealthcareApiClient client;
+
+      /**
+       * Instantiates a new Write HL7v2 fn.
+       *
+       * @param hl7v2Store the HL7v2 store
+       * @param writeMethod the write method
+       */
+      WriteHL7v2Fn(String hl7v2Store, Write.WriteMethod writeMethod) {
+        this.hl7v2Store = hl7v2Store;
+        this.writeMethod = writeMethod;
+      }
+
+      /**
+       * Init client.
+       *
+       * @throws IOException the io exception
+       */
+      @Setup
+      public void initClient() throws IOException {
+        this.client = new HttpHealthcareApiClient();
+      }
+
+      /**
+       * Write messages.
+       *
+       * @param context the context
+       */
+      @ProcessElement
+      public void writeMessages(ProcessContext context) {
+        HL7v2Message msg = context.element();
+        long startTime = System.currentTimeMillis();
+        Sleeper sleeper = Sleeper.DEFAULT;
+        switch (writeMethod) {
+          case BATCH_IMPORT:
+            throw new UnsupportedOperationException("The Batch import API is not available yet");
+          case INGEST:
+          default:
+            try {
+              client.ingestHL7v2Message(hl7v2Store, msg.toModel());
 
 Review comment:
   This section of the code may represent a performance blocker, as we're issuing a single blocking call per message. This is fine, but you may eventually want to batch messages to send for ingestion. It also depends of whether the service supports batching.
   
   We do something like that for BigQuery streaming inserts: https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java#L85-L156
   
   We send a list of messages [m1, m2, m3, ...], and if there are any errors, BQ also returns a list of errors that can be mapped to individual messages.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-603573376
 
 
   @pabloem sorry for the rage commits today.
   This is now ready for review.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612259919
 
 
   @lukecwik would it be a reasonable compromise to introduce a `google_oauth_clients_verion`? If not I believe we are at an impass.
   Reasoning / Research / Background:
   - features in healthcare API making this PR possible require >=1.30.9 (healthcare api absolute minimum version is 1.30.8)
   - all deps under `com.google.apis` are updatable to 1.30.9
   - the [two](https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client/) [deps](https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client-java6/) under `com.google.oauth-client` have versions that stop at 1.30.6
   - I have a suspicion that this 1.30.9 for `com.google.apis` is not even really related to the 1.30.6 for `com.google.oauth-clients` but an unfortunate naming convention similarity
       - see that 1.30.6 [oauth client updated fairly recently on 2020-03-05](https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client/1.30.6/) and is required by the version of [bigdata.oss:gcsio 2.1.2](https://mvnrepository.com/artifact/com.google.cloud.bigdataoss/gcsio/2.1.2) we just updated to in 5208c3d. 
   
   I've prepared this in the latest commit 
   59dc3c2  to make this a concrete proposal to review / run pre/post commits on.
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-603460527
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614260458
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614269167
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] brianlucier commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
brianlucier commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-611591653
 
 
   > 3\. What is the feasibility / process for back-porting this to prior beam releases for expediting use by users who want to use an already released version e.g. 2.19 ?
   
   bumping this question.  would like to know timing on when I can see it in a released version... either backport, or a new one.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614169385
 
 
   @lukecwik please let me know if there's anything else I can do to give confidence in the dependency changes.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r394520512
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   Why is this not `google_clients_version`?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612237090
 
 
   Note that your dependency report seems to be showing that you should be able to update to the version needed by the healthcare lib.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] suztomo commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
suztomo commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r409058454
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -469,6 +470,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_cloud_datastore_v1_proto_client      : "com.google.cloud.datastore:datastore-v1-proto-client:1.6.3",
         google_cloud_spanner                        : "com.google.cloud:google-cloud-spanner:$google_cloud_spanner_version",
         google_http_client                          : "com.google.http-client:google-http-client:$google_http_clients_version",
+        google_http_client_gson                     : "com.google.http-client:google-http-client-gson:$google_http_clients_version",
 
 Review comment:
   Do you really need google_http_client_gson here? Beam already has google_http_client_jackson2 and it seems CloudHealthcare.Builder accepts Jackson.
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612259919
 
 
   @lukecwik would it be a reasonable compromise to introduce a `google_oauth_clients_verion`? If not I believe we are at an impass.
   Reasoning / Research / Background:
   - features in healthcare API making this PR possible require >=1.30.9 (healthcare api absolute minimum version is 1.30.8)
   - all deps under `com.google.apis` are updatable to 1.30.9
   - the [two](https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client/) [deps](https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client-java6/) under `com.google.oauth-client` have versions that stop at 1.30.6
   - I have a suspicion that this 1.30.9 for `com.google.apis` is not even really related to the 1.30.6 for `com.google.oauth-clients` but an unfortunate naming convention similarity
       - see that 1.30.6 [oauth client updated fairly recently on 2020-03-05](https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client/1.30.6/)
   
   I've prepared this in the latest commit 
   59dc3c2  to make this a concrete proposal to review / run pre/post commits on.
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404366818
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOErrorCoder.java
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * 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 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;
+
+public class HealthcareIOErrorCoder<T> extends CustomCoder<HealthcareIOError<T>> {
+  private final Coder<T> originalCoder;
+  private static final NullableCoder<String> STRING_CODER = NullableCoder.of(StringUtf8Coder.of());
+
+  HealthcareIOErrorCoder(Coder<T> originalCoder) {
+    this.originalCoder = NullableCoder.of(originalCoder);
+  }
+
+  @Override
+  public void encode(HealthcareIOError<T> value, OutputStream outStream)
+      throws CoderException, IOException {
+
+    originalCoder.encode(value.getDataResource(), outStream);
+
 
 Review comment:
   Good Catch.
   We should serialize / deserialize observed time.
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614311975
 
 
   @pabloem please do not retest this until I say so. 
   reshuffle is messing up something to do w/ coders in my ITs.
   will investigate and let you know when it's safe to re run tests

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614340970
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614263256
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613120004
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612260628
 
 
   retest this please
   
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r406328992
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   However the generated README has version listed as v1beta1-rev20200327-1.30.9
   so maybe I am misunderstanding.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612501538
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404357718
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,597 @@
+/*
+ * 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.v1beta1.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.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 can be fetched from the HL7v2 store in two ways Message Fetching and Message
+ * Listing.
+ *
+ * <p>Message Fetching
+ *
+ * <p>Message Fetching with {@link HL7v2IO.Read} supports use cases where you have a ${@link
+ * PCollection} of message IDS. This is appropriate for reading the HL7v2 notifications from a
+ * Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually
+ * prepared list of messages that you need to process (e.g. in a text file read with {@link
+ * org.apache.beam.sdk.io.TextIO}) .
+ *
+ * <p>Fetch Message contents from HL7v2 Store based on the {@link PCollection} of message ID strings
+ * {@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},
+ * this can be used to write to the dead letter storage system of your choosing. This error handling
+ * is mainly to catch scenarios where the upstream {@link PCollection} contains IDs that are not
+ * valid or are not reachable due to permissions issues.
+ *
+ * <p>Message Listing Message Listing with {@link HL7v2IO.ListHL7v2Messages} supports batch use
+ * cases where you want to process all the messages in an HL7v2 store or those matching a
+ * filter @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters</a>
+ * This paginates through results of a Messages.List call @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list</a>
+ * and outputs directly to a {@link PCollection} of {@link HL7v2Message}. In these use cases, the
+ * error handling similar to above is unnecessary because we are listing from the source of truth
+ * the pipeline should fail transparently if this transform fails to paginate through all the
+ * results.
+ *
+ * <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} containing the {@link HL7v2Message}
+ * 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 Read Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * PCollection<HL7v2Message> out = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2Messages(
+ *           Collections.singletonList(options.getInputHL7v2Store()), option.getHL7v2Filter()))
+ *    // Go about your happy path transformations.
+ *   .apply("Process HL7v2 Messages", ...);
+ * 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) {
 
 Review comment:
   This may be a matter of style, but `ingestMessages` makes me think of ingesting them *into* the pipeline - though perhaps users will have a clear context when they write their pipelines.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r401826527
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOWriteIT.java
 ##########
 @@ -0,0 +1,72 @@
+/*
+ * 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 static org.apache.beam.sdk.io.gcp.healthcare.HL7v2IOTestUtil.MESSAGES;
+import static org.apache.beam.sdk.io.gcp.healthcare.HL7v2IOTestUtil.deleteAllHL7v2Messages;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class HL7v2IOWriteIT {
+
+  private HL7v2IOTestOptions options;
+  private transient HealthcareApiClient client;
+
+  @Before
+  public void setup() throws Exception {
+    if (client == null) {
+      client = new HttpHealthcareApiClient();
+    }
+    PipelineOptionsFactory.register(HL7v2IOTestOptions.class);
+    options = TestPipeline.testingPipelineOptions().as(HL7v2IOTestOptions.class);
+    options.setHL7v2Store(
+        "projects/jferriero-dev/locations/us-central1/datasets/raw-dataset/hl7V2Stores/jake-hl7");
 
 Review comment:
   Does this test actually write to that HL7vStore? If so, can multiple tests run at the same time? - it's not necessary, but I'd like to clarify.
   
   We may also be able to provision something on the apache-beam-testing GCP project, if it's not very high-maintenance : )

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r406330770
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   It'd make this PR cleaner to use this most recent version because [this commit adds `SchematizedData`](https://github.com/googleapis/google-api-java-client-services/commit/48db535eb30fea17c615f9951d7afad6f3ea72b4#diff-89048b80fe6bcee9b0aade84024aea66).
   which I am currently working around [here](https://github.com/apache/beam/pull/11151/files#diff-2173a334dc15f8b7dc3257efeec64504R27).

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612268084
 
 
   now linkage checker is giving me no output:
   ```
   jferriero@shadow-gallery:~/VersionControl/beam$ sdks/java/build-tools/beam-linkage-check.sh
   Fri 10 Apr 2020 04:42:32 PM PDT: Installing artifacts of HL7v2IO(785a5937) to Maven local repository.
   jferriero@shadow-gallery:~/VersionControl/beam$ 
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r406860029
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   Typically knowing which versions you can try is very hard without a lot of experience with a specific set of dependencies (e.g. knowing a lot about jackson echo system vs gRPC vs ...).
   
   You should find a google-cloud-bom and matching libraries-bom that contain the healthcare dep you want to add. That should give you hints as to what are all the versions of things that work together. You will still need to perform the linkage checker analysis and run the tests as described above.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613681105
 
 
   @lukecwik ptal so @jaketf won't have to rebase if changes look fine

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r398858533
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java
 ##########
 @@ -0,0 +1,452 @@
+/*
+ * 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.client.http.HttpHeaders;
+import com.google.api.client.http.HttpRequest;
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.client.http.javanet.NetHttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.client.util.ArrayMap;
+import com.google.api.services.healthcare.v1alpha2.CloudHealthcare;
+import com.google.api.services.healthcare.v1alpha2.CloudHealthcare.Projects.Locations.Datasets.Hl7V2Stores.Messages;
+import com.google.api.services.healthcare.v1alpha2.CloudHealthcareScopes;
+import com.google.api.services.healthcare.v1alpha2.model.CreateMessageRequest;
+import com.google.api.services.healthcare.v1alpha2.model.Empty;
+import com.google.api.services.healthcare.v1alpha2.model.Hl7V2Store;
+import com.google.api.services.healthcare.v1alpha2.model.HttpBody;
+import com.google.api.services.healthcare.v1alpha2.model.IngestMessageRequest;
+import com.google.api.services.healthcare.v1alpha2.model.IngestMessageResponse;
+import com.google.api.services.healthcare.v1alpha2.model.ListMessagesResponse;
+import com.google.api.services.healthcare.v1alpha2.model.Message;
+import com.google.api.services.healthcare.v1alpha2.model.SearchResourcesRequest;
+import com.google.auth.oauth2.GoogleCredentials;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URI;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Spliterator;
+import java.util.Spliterators;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.gcp.util.RetryHttpRequestInitializer;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
+
+/**
+ * A client that talks to the Cloud Healthcare API through HTTP requests. This client is created
+ * mainly to encapsulate the unserializable dependencies, since most generated classes are not
+ * serializable in the HTTP client.
+ */
+public class HttpHealthcareApiClient<T> implements HealthcareApiClient, Serializable {
+
+  private transient CloudHealthcare client;
+
+  /**
+   * Instantiates a new Http healthcare api client.
+   *
+   * @throws IOException the io exception
+   */
+  public HttpHealthcareApiClient() throws IOException {
+    initClient();
+  }
+
+  /**
+   * Instantiates a new Http healthcare api client.
+   *
+   * @param client the client
+   * @throws IOException the io exception
+   */
+  public HttpHealthcareApiClient(CloudHealthcare client) throws IOException {
+    this.client = client;
+    initClient();
+  }
+
+  @VisibleForTesting
+  static <T, X extends Collection<T>> Stream<T> flattenIteratorCollectionsToStream(
+      Iterator<X> iterator) {
+    Spliterator<Collection<T>> spliterator = Spliterators.spliteratorUnknownSize(iterator, 0);
+    return StreamSupport.stream(spliterator, false).flatMap(Collection::stream);
+  }
+
+  public JsonFactory getJsonFactory() {
+    return this.client.getJsonFactory();
+  }
+
+  @Override
+  public ListMessagesResponse makeHL7v2ListRequest(
+      String hl7v2Store, @Nullable String filter, @Nullable String pageToken) throws IOException {
+
+    Messages.List baseRequest =
+        client
+            .projects()
+            .locations()
+            .datasets()
+            .hl7V2Stores()
+            .messages()
+            .list(hl7v2Store)
+            .setPageToken(pageToken);
+
+    if (Strings.isNullOrEmpty(filter)) {
+      return baseRequest.execute();
+    } else {
+      return baseRequest.setFilter(filter).execute();
+    }
+  }
+
+  /**
+   * Gets message id page iterator.
+   *
+   * @param hl7v2Store the HL7v2 store
+   * @return the message id page iterator
+   * @throws IOException the io exception
+   */
+  @Override
+  public Stream<String> getHL7v2MessageIDStream(String hl7v2Store) throws IOException {
+    return getHL7v2MessageIDStream(hl7v2Store, null);
+  }
+
+  /**
+   * Get a {@link Stream} of message IDs from flattening the pages of a new {@link
+   * HL7v2MessageIDPages}.
+   *
+   * @param hl7v2Store the HL7v2 store
+   * @param filter the filter
+   * @return the message id Stream
+   * @throws IOException the io exception
+   */
+  @Override
+  public Stream<String> getHL7v2MessageIDStream(String hl7v2Store, @Nullable String filter)
+      throws IOException {
+    Iterator<List<String>> iterator = new HL7v2MessageIDPages(this, hl7v2Store, filter).iterator();
+    return flattenIteratorCollectionsToStream(iterator);
+  }
+
+  /**
+   * Gets HL7v2 message.
+   *
+   * @param msgName the msg name
+   * @return the message
+   * @throws IOException the io exception
+   * @throws ParseException the parse exception
+   */
+  @Override
+  public Message getHL7v2Message(String msgName) throws IOException {
+    Message msg =
+        client.projects().locations().datasets().hl7V2Stores().messages().get(msgName).execute();
+    if (msg == null) {
+      throw new IOException(String.format("Couldn't find message: %s.", msgName));
+    }
+    return msg;
+  }
+
+  @Override
+  public Empty deleteHL7v2Message(String msgName) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .delete(msgName)
+        .execute();
+  }
+
+  /**
+   * Gets HL7v2 store.
+   *
+   * @param storeName the store name
+   * @return the HL7v2 store
+   * @throws IOException the io exception
+   */
+  @Override
+  public Hl7V2Store getHL7v2Store(String storeName) throws IOException {
+    return client.projects().locations().datasets().hl7V2Stores().get(storeName).execute();
+  }
+
+  @Override
+  public IngestMessageResponse ingestHL7v2Message(String hl7v2Store, Message msg)
+      throws IOException {
+    IngestMessageRequest ingestMessageRequest = new IngestMessageRequest();
+    ingestMessageRequest.setMessage(msg);
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .ingest(hl7v2Store, ingestMessageRequest)
+        .execute();
+  }
 
 Review comment:
   yes [here](https://github.com/apache/beam/pull/11151/files/cac03ec30c77e6bd45954e659b875d0378370892#diff-3ab3eba677d52f20f1a688b1017adf84R274) we extend [`RetryHttpRequestInitializer`](https://beam.apache.org/releases/javadoc/2.13.0/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializer.html)

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612259919
 
 
   @lukecwik would it be a reasonable compromise to introduce a `google_oauth_clients_verion`? If not I believe we are at an impass.
   Reasoning / Research / Background:
   - features in healthcare API making this PR possible require >=1.30.9 (healthcare api absolute minimum version is 1.30.8)
   - all deps under `com.google.apis` are updatable to 1.30.9
   - the [two](https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client/) [deps](https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client-java6/) under `com.google.oauth-client` have versions that stop at 1.30.6
   - I have a suspicion that this 1.30.9 for `com.google.apis` is not even really related to the 1.30.6 for `com.google.oauth-clients` but an unfortunate naming convention similarity
       - see that 1.30.6 [oauth client updated fairly recently on 2020-03-05](https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client/1.30.6/)
   
   I've prepared this in the latest commit to make this a concrete proposal to review / run pre/post commits on.
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-610050634
 
 
   Next Steps (based on offline feedback):
   - [x] Improve API for users:
       - [x] Add static methods for common patterns with `ListHL7v2Messages` 
       - [x] Add `ValueProvider<String>` support to ease use in the DataflowTemplates
           - [x] `ListHL7v2Messages` (hl7v2Store and filter)
           - [x] `Write` (hl7v2Store)
   - [ ] "standardize" integration tests
       - [x] Refactor ITs to create / destroy HL7v2 Store under a parameterized dataset in `@BeforeClass` `@AfterClass` to avoid issues with parallel tests runs.
       - [x] Remove hard coding of my HL7v2Store / project in integration tests.
       - [ ] Add Healthcare API Dataset to Beam integration test project (pending permissions in [this dev list thread](https://lists.apache.org/thread.html/rebe5cd40a40a9fc7f2c1d563b48ee1ce4ff9cac3dfdc0258006cc686%40%3Cdev.beam.apache.org%3E))

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612260585
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r394632407
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,620 @@
+/*
+ * 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.IngestMessageResponse;
+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.Collections;
+import java.util.List;
+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.PDone;
+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.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 HL7v2 Messages are fetched from the HL7v2 store based on the {@link PCollection} of of
+ * message IDs {@link String}s produced by the {@link AutoValue_HL7v2IO_Read#getMessageIDTransform}
+ * as {@link PCollectionTuple}*** containing an {@link HL7v2IO.Read#OUT} tag for successfully
+ * fetched messages and a {@link HL7v2IO.Read#DEAD_LETTER} tag for message IDs that could not be
+ * fetched.
+ *
+ * <p>HL7v2 stores can be read in several ways: - Unbounded: based on the Pub/Sub Notification
+ * Channel {@link HL7v2IO#readNotificationSubscription(String)} - Bounded: based on reading an
+ * entire HL7v2 store (or stores) {@link HL7v2IO#readHL7v2Store(String)} - Bounded: based on reading
+ * an HL7v2 store with a filter
+ *
+ * <p>Note, due to the flexibility of this Read transform, this must output a dead letter queue.
+ * This handles the scenario where the the PTransform that populates a PCollection of message IDs
+ * contains message IDs that do not exist in the HL7v2 stores.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline pipeline = Pipeline.create(options)
+ *
+ *
+ * PCollectionTuple messages = pipeline.apply(
+ *     new HLv2IO.readNotifications(options.getNotificationSubscription())
 
 Review comment:
   I believe you don't need the `new` here.
   ```suggestion
    *     HLv2IO.readNotifications(options.getNotificationSubscription())
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-612234407
 
 
   > @lukecwik I ran the linkeage check "as-is" ([log](https://gist.github.com/suztomo/3178f298bd315e2cc3897b5528a961b8)) and IIUC this has no issues.
   > 
   > Is this acceptable or is it a hard requirement that I update google_clients_version?
   
   This is a hard requirement. Either we are using the google_clients_version of shared deps or google_healthcare_clients_version for deps. We can't be using both.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lukecwik commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r406476510
 
 

 ##########
 File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
 ##########
 @@ -454,6 +455,7 @@ class BeamModulePlugin implements Plugin<Project> {
         google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20191003-$google_clients_version",
         google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20191206-$google_clients_version",
         google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1alpha2-rev20190901-$google_cloud_healthcare_version",
 
 Review comment:
   It doesn't have to be 1.30.3, it could be any later version. The purpose of them being the same is to ensure that GCP dependencies are compatible with each other so the healthcare dep doesn't break other GCP deps and vice versa.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r404371522
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
 ##########
 @@ -0,0 +1,597 @@
+/*
+ * 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.v1beta1.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.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 can be fetched from the HL7v2 store in two ways Message Fetching and Message
+ * Listing.
+ *
+ * <p>Message Fetching
+ *
+ * <p>Message Fetching with {@link HL7v2IO.Read} supports use cases where you have a ${@link
+ * PCollection} of message IDS. This is appropriate for reading the HL7v2 notifications from a
+ * Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually
+ * prepared list of messages that you need to process (e.g. in a text file read with {@link
+ * org.apache.beam.sdk.io.TextIO}) .
+ *
+ * <p>Fetch Message contents from HL7v2 Store based on the {@link PCollection} of message ID strings
+ * {@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},
+ * this can be used to write to the dead letter storage system of your choosing. This error handling
+ * is mainly to catch scenarios where the upstream {@link PCollection} contains IDs that are not
+ * valid or are not reachable due to permissions issues.
+ *
+ * <p>Message Listing Message Listing with {@link HL7v2IO.ListHL7v2Messages} supports batch use
+ * cases where you want to process all the messages in an HL7v2 store or those matching a
+ * filter @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters</a>
+ * This paginates through results of a Messages.List call @see <a
+ * href=>https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list</a>
+ * and outputs directly to a {@link PCollection} of {@link HL7v2Message}. In these use cases, the
+ * error handling similar to above is unnecessary because we are listing from the source of truth
+ * the pipeline should fail transparently if this transform fails to paginate through all the
+ * results.
+ *
+ * <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} containing the {@link HL7v2Message}
+ * 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 Read Example:
+ *
+ * <pre>{@code
+ * PipelineOptions options = ...;
+ * Pipeline p = Pipeline.create(options);
+ *
+ * PCollection<HL7v2Message> out = p
+ *   .apply(
+ *       "List messages in HL7v2 store with filter",
+ *       ListHL7v2Messages(
+ *           Collections.singletonList(options.getInputHL7v2Store()), option.getHL7v2Filter()))
+ *    // Go about your happy path transformations.
+ *   .apply("Process HL7v2 Messages", ...);
+ * 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 ListHL7v2Messages} 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} message IDs which failed to be fetched from the HL7v2 store, with
+     *       error message and stacktrace.
+     * </ul>
+     */
+    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 successfulHL7v2MessageGets =
+            Metrics.counter(
+                FetchHL7v2Message.HL7v2MessageGetFn.class, "successful-hl7v2-message-gets");
+        private HealthcareApiClient client;
+
+        /** 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();
+        }
+
+        /**
+         * 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 {
+          long startTime = System.currentTimeMillis();
+
+          com.google.api.services.healthcare.v1beta1.model.Message msg =
 
 Review comment:
   API only supports [get](https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/get) an individual resource or [list](https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.hl7V2Stores.messages/list) resources w/ a filter.
   
   One day if we could requests a list of message IDs that would be a great optimization!
   @lastomato do you know if HL7 search or any other roadmap for API to support this?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] chamikaramj commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-615300431
 
 
   Seems like one of the new tests is flaky.
   
   https://builds.apache.org/job/beam_PostCommit_Java/5947/
   https://builds.apache.org/job/beam_PostCommit_Java/5943/
   https://builds.apache.org/job/beam_PostCommit_Java/5942/

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r398215206
 
 

 ##########
 File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java
 ##########
 @@ -0,0 +1,452 @@
+/*
+ * 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.client.http.HttpHeaders;
+import com.google.api.client.http.HttpRequest;
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.client.http.javanet.NetHttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.client.util.ArrayMap;
+import com.google.api.services.healthcare.v1alpha2.CloudHealthcare;
+import com.google.api.services.healthcare.v1alpha2.CloudHealthcare.Projects.Locations.Datasets.Hl7V2Stores.Messages;
+import com.google.api.services.healthcare.v1alpha2.CloudHealthcareScopes;
+import com.google.api.services.healthcare.v1alpha2.model.CreateMessageRequest;
+import com.google.api.services.healthcare.v1alpha2.model.Empty;
+import com.google.api.services.healthcare.v1alpha2.model.Hl7V2Store;
+import com.google.api.services.healthcare.v1alpha2.model.HttpBody;
+import com.google.api.services.healthcare.v1alpha2.model.IngestMessageRequest;
+import com.google.api.services.healthcare.v1alpha2.model.IngestMessageResponse;
+import com.google.api.services.healthcare.v1alpha2.model.ListMessagesResponse;
+import com.google.api.services.healthcare.v1alpha2.model.Message;
+import com.google.api.services.healthcare.v1alpha2.model.SearchResourcesRequest;
+import com.google.auth.oauth2.GoogleCredentials;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URI;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Spliterator;
+import java.util.Spliterators;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.gcp.util.RetryHttpRequestInitializer;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
+
+/**
+ * A client that talks to the Cloud Healthcare API through HTTP requests. This client is created
+ * mainly to encapsulate the unserializable dependencies, since most generated classes are not
+ * serializable in the HTTP client.
+ */
+public class HttpHealthcareApiClient<T> implements HealthcareApiClient, Serializable {
+
+  private transient CloudHealthcare client;
+
+  /**
+   * Instantiates a new Http healthcare api client.
+   *
+   * @throws IOException the io exception
+   */
+  public HttpHealthcareApiClient() throws IOException {
+    initClient();
+  }
+
+  /**
+   * Instantiates a new Http healthcare api client.
+   *
+   * @param client the client
+   * @throws IOException the io exception
+   */
+  public HttpHealthcareApiClient(CloudHealthcare client) throws IOException {
+    this.client = client;
+    initClient();
+  }
+
+  @VisibleForTesting
+  static <T, X extends Collection<T>> Stream<T> flattenIteratorCollectionsToStream(
+      Iterator<X> iterator) {
+    Spliterator<Collection<T>> spliterator = Spliterators.spliteratorUnknownSize(iterator, 0);
+    return StreamSupport.stream(spliterator, false).flatMap(Collection::stream);
+  }
+
+  public JsonFactory getJsonFactory() {
+    return this.client.getJsonFactory();
+  }
+
+  @Override
+  public ListMessagesResponse makeHL7v2ListRequest(
+      String hl7v2Store, @Nullable String filter, @Nullable String pageToken) throws IOException {
+
+    Messages.List baseRequest =
+        client
+            .projects()
+            .locations()
+            .datasets()
+            .hl7V2Stores()
+            .messages()
+            .list(hl7v2Store)
+            .setPageToken(pageToken);
+
+    if (Strings.isNullOrEmpty(filter)) {
+      return baseRequest.execute();
+    } else {
+      return baseRequest.setFilter(filter).execute();
+    }
+  }
+
+  /**
+   * Gets message id page iterator.
+   *
+   * @param hl7v2Store the HL7v2 store
+   * @return the message id page iterator
+   * @throws IOException the io exception
+   */
+  @Override
+  public Stream<String> getHL7v2MessageIDStream(String hl7v2Store) throws IOException {
+    return getHL7v2MessageIDStream(hl7v2Store, null);
+  }
+
+  /**
+   * Get a {@link Stream} of message IDs from flattening the pages of a new {@link
+   * HL7v2MessageIDPages}.
+   *
+   * @param hl7v2Store the HL7v2 store
+   * @param filter the filter
+   * @return the message id Stream
+   * @throws IOException the io exception
+   */
+  @Override
+  public Stream<String> getHL7v2MessageIDStream(String hl7v2Store, @Nullable String filter)
+      throws IOException {
+    Iterator<List<String>> iterator = new HL7v2MessageIDPages(this, hl7v2Store, filter).iterator();
+    return flattenIteratorCollectionsToStream(iterator);
+  }
+
+  /**
+   * Gets HL7v2 message.
+   *
+   * @param msgName the msg name
+   * @return the message
+   * @throws IOException the io exception
+   * @throws ParseException the parse exception
+   */
+  @Override
+  public Message getHL7v2Message(String msgName) throws IOException {
+    Message msg =
+        client.projects().locations().datasets().hl7V2Stores().messages().get(msgName).execute();
+    if (msg == null) {
+      throw new IOException(String.format("Couldn't find message: %s.", msgName));
+    }
+    return msg;
+  }
+
+  @Override
+  public Empty deleteHL7v2Message(String msgName) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .delete(msgName)
+        .execute();
+  }
+
+  /**
+   * Gets HL7v2 store.
+   *
+   * @param storeName the store name
+   * @return the HL7v2 store
+   * @throws IOException the io exception
+   */
+  @Override
+  public Hl7V2Store getHL7v2Store(String storeName) throws IOException {
+    return client.projects().locations().datasets().hl7V2Stores().get(storeName).execute();
+  }
+
+  @Override
+  public IngestMessageResponse ingestHL7v2Message(String hl7v2Store, Message msg)
+      throws IOException {
+    IngestMessageRequest ingestMessageRequest = new IngestMessageRequest();
+    ingestMessageRequest.setMessage(msg);
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .ingest(hl7v2Store, ingestMessageRequest)
+        .execute();
+  }
+
+  @Override
+  public HttpBody fhirSearch(String fhirStore, SearchResourcesRequest query) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .search(fhirStore, query)
+        .execute();
+  }
+
+  @Override
+  public Message createHL7v2Message(String hl7v2Store, Message msg) throws IOException {
+    CreateMessageRequest createMessageRequest = new CreateMessageRequest();
+    createMessageRequest.setMessage(msg);
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .hl7V2Stores()
+        .messages()
+        .create(hl7v2Store, createMessageRequest)
+        .execute();
+  }
+
+  @Override
+  public HttpBody createFhirResource(String fhirStore, String type, HttpBody body)
+      throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .create(fhirStore, type, body)
+        .execute();
+  }
+
+  @Override
+  public HttpBody executeFhirBundle(String fhirStore, HttpBody bundle) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .executeBundle(fhirStore, bundle)
+        .execute();
+  }
+
+  @Override
+  public HttpBody listFHIRResourceForPatient(String fhirStore, String patient) throws IOException {
+    return client
+        .projects()
+        .locations()
+        .datasets()
+        .fhirStores()
+        .fhir()
+        .patientEverything(patient)
+        .execute();
+  }
+
+  @Override
+  public HttpBody readFHIRResource(String fhirStore, String resource) throws IOException {
+    return client.projects().locations().datasets().fhirStores().fhir().read(resource).execute();
+  }
+
+  private static class AuthenticatedRetryInitializer extends RetryHttpRequestInitializer {
+    GoogleCredentials credentials;
+
+    public AuthenticatedRetryInitializer(GoogleCredentials credentials) {
+      super();
+      this.credentials = credentials;
+    }
+
+    @Override
+    public void initialize(HttpRequest request) throws IOException {
+      super.initialize(request);
+      if (!credentials.hasRequestMetadata()) {
+        return;
+      }
+      HttpHeaders requestHeaders = request.getHeaders();
+      requestHeaders.setUserAgent("apache-beam-hl7v2-io");
+      URI uri = null;
+      if (request.getUrl() != null) {
+        uri = request.getUrl().toURI();
+      }
+      Map<String, List<String>> credentialHeaders = credentials.getRequestMetadata(uri);
+      if (credentialHeaders == null) {
+        return;
+      }
+      for (Map.Entry<String, List<String>> entry : credentialHeaders.entrySet()) {
+        String headerName = entry.getKey();
+        List<String> requestValues = new ArrayList<>(entry.getValue());
+        requestHeaders.put(headerName, requestValues);
+      }
+    }
+  }
+
+  private void initClient() throws IOException {
+    // Create a HttpRequestInitializer, which will provide a baseline configuration to all requests.
+    // HttpRequestInitializer requestInitializer = new RetryHttpRequestInitializer();
+    // GoogleCredentials credentials = GoogleCredentials.getApplicationDefault();
+    HttpRequestInitializer requestInitializer =
+        new AuthenticatedRetryInitializer(
+            GoogleCredentials.getApplicationDefault()
+                .createScoped(CloudHealthcareScopes.CLOUD_PLATFORM));
+
+    client =
+        new CloudHealthcare.Builder(new NetHttpTransport(), new GsonFactory(), requestInitializer)
+            .setApplicationName("apache-beam-hl7v2-io")
+            .build();
+  }
+
+  public static class HL7v2MessageIDPages implements Iterable<List<String>> {
+
+    private final String hl7v2Store;
+    private final String filter;
+    private transient HealthcareApiClient client;
+
+    /**
+     * Instantiates a new HL7v2 message id pages.
+     *
+     * @param client the client
+     * @param hl7v2Store the HL7v2 store
+     */
+    HL7v2MessageIDPages(HealthcareApiClient client, String hl7v2Store) {
+      this.client = client;
+      this.hl7v2Store = hl7v2Store;
+      this.filter = null;
+    }
+
+    /**
+     * Instantiates a new HL7v2 message id pages.
+     *
+     * @param client the client
+     * @param hl7v2Store the HL7v2 store
+     * @param filter the filter
+     */
+    HL7v2MessageIDPages(HealthcareApiClient client, String hl7v2Store, @Nullable String filter) {
+      this.client = client;
+      this.hl7v2Store = hl7v2Store;
+      this.filter = filter;
+    }
+
+    /**
+     * Make list request list messages response.
+     *
+     * @param client the client
+     * @param hl7v2Store the hl 7 v 2 store
+     * @param filter the filter
+     * @param pageToken the page token
+     * @return the list messages response
+     * @throws IOException the io exception
+     */
+    public static ListMessagesResponse makeListRequest(
+        HealthcareApiClient client,
+        String hl7v2Store,
+        @Nullable String filter,
+        @Nullable String pageToken)
+        throws IOException {
+      return client.makeHL7v2ListRequest(hl7v2Store, filter, pageToken);
+    }
+
+    @Override
+    public Iterator<List<String>> iterator() {
+      return new HL7v2MessageIDPagesIterator(this.client, this.hl7v2Store, this.filter);
+    }
+
+    /** The type Hl7v2 message id pages iterator. */
+    public static class HL7v2MessageIDPagesIterator implements Iterator<List<String>> {
+
+      private final String hl7v2Store;
+      private final String filter;
+      private HealthcareApiClient client;
+      private String pageToken;
+      private boolean isFirstRequest;
+
+      /**
+       * Instantiates a new Hl 7 v 2 message id pages iterator.
+       *
+       * @param client the client
+       * @param hl7v2Store the hl 7 v 2 store
+       * @param filter the filter
+       */
+      HL7v2MessageIDPagesIterator(
+          HealthcareApiClient client, String hl7v2Store, @Nullable String filter) {
+        this.client = client;
+        this.hl7v2Store = hl7v2Store;
+        this.filter = filter;
+        this.pageToken = null;
+        this.isFirstRequest = true;
+      }
+
+      @Override
+      public boolean hasNext() throws NoSuchElementException {
+        if (isFirstRequest) {
+          try {
+            ListMessagesResponse response = makeListRequest(client, hl7v2Store, filter, pageToken);
+            List<String> msgs = response.getMessages();
+            if (msgs == null) {
+              if (response.get("hl7V2Messages") != null) {
+                return ((ArrayList<ArrayMap<String, String>>) response.get("hl7V2Messages")).size()
+                    > 0;
+              }
+              return false;
+            } else {
+              return !msgs.isEmpty();
+            }
+          } catch (IOException e) {
+            throw new NoSuchElementException(
+                String.format(
+                    "Failed to list first page of HL7v2 messages from %s: %s",
+                    hl7v2Store, e.getMessage()));
+          }
+        }
+        return this.pageToken != null;
+      }
+
+      @Override
+      public List<String> next() throws NoSuchElementException {
+        try {
+          ListMessagesResponse response = makeListRequest(client, hl7v2Store, filter, pageToken);
+          this.isFirstRequest = false;
+          this.pageToken = response.getNextPageToken();
+          List<String> msgs = response.getMessages();
+          if (msgs == null && response.get("hl7V2Messages") != null) {
 
 Review comment:
   Note, this `ArrayMap<String, String>` is basically the `Message` model.
   If you use the alpha API to query an beta HL7v2 store, there will be a single key `name` with the message ID value.
   If you use the alpha API to query an alpha HL7v2 store, there will be all of the keys in the `Message` model.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-613059073
 
 
   Run Java PostCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-604001261
 
 
   Future improvements:
   - Currently this uses the alpha API because the motivating use case of HL7v2 -> FHIR mapping which requires the `schematizedData` field (not yet available in beta API).
   - ~For backwards compatibility the `ListHL7v2MessageIDs` always uses Messages.List to get message IDs and relies on the the `HL7v2IO.Read` to fetch the message contents. This provides a consistent flow for real-time and batch and provides backwards compatibility. However, in the alpha API Messages.List returns the actual message contents (rather than just the message IDs). This leads to us reading the messages contents twice for alpha HL7v2 stores. I suggest we address optimizing this "double fetch" in a future PR as the alpha API stabilizes.~

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] pabloem commented on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-614268998
 
 
   Run Java PreCommit

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lastomato edited a comment on issue #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
lastomato edited a comment on issue #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#issuecomment-606940440
 
 
   Sorry for the delay. Please see my comments inline.
   
   > Open Questions:
   > 
   > 1. Should we remove adaptive throttling?
   
   I think it is fine to remove it since the API has quota enabled by default, and retry logic is in place.
   
   >    
   >    * Seems that we're using retries in the client request initializer and right now a "bad record" will slow down the Read / Write (even though the error has nothing to do with the HL7v2 store being overwhelmed). Originally we wanted to be safe with overwhelming QPS on the HL7v2 store in batch scenarios.
   > 2. Should we add more to the `HealthcareIOError`?
   
   The second point below would be very helpful, but I am fine with adding it to next PR (if that's easier).
   
   >    
   >    * Add (processing time) Timestamp?
   >    * Add a convenience DoFn `HealthcareIOErrrorToTableRowFn` to ease writing deadletter queue to BigQuery.
   > 3. Would it be more useful to expose an error rate metric than an error count?
   
   This functionality is probably already provided by services like stackdriver, we might want to wait until there is a concrete use case.
   
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] jaketf commented on a change in pull request #11151: [BEAM-9468] Hl7v2 io

Posted by GitBox <gi...@apache.org>.
jaketf commented on a change in pull request #11151: [BEAM-9468]  Hl7v2 io
URL: https://github.com/apache/beam/pull/11151#discussion_r399452501
 
 

 ##########
 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:
   This can be resolved because beta API also supports view now.
   Refactored to serve this use case w/ a separate PTransform and updated docstring.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services