[ https://issues.apache.org/jira/browse/BEAM-9468?focusedWorklogId=409832&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-409832 ]
ASF GitHub Bot logged work on BEAM-9468: ---------------------------------------- Author: ASF GitHub Bot Created on: 25/Mar/20 21:28 Start Date: 25/Mar/20 21:28 Worklog Time Spent: 10m Work Description: brianlucier commented on 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: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 409832) Time Spent: 5h (was: 4h 50m) > Add Google Cloud Healthcare API IO Connectors > --------------------------------------------- > > Key: BEAM-9468 > URL: https://issues.apache.org/jira/browse/BEAM-9468 > Project: Beam > Issue Type: New Feature > Components: io-java-gcp > Reporter: Jacob Ferriero > Assignee: Jacob Ferriero > Priority: Minor > Time Spent: 5h > Remaining Estimate: 0h > > Add IO Transforms for the HL7v2, FHIR and DICOM stores in the [Google Cloud > Healthcare API|https://cloud.google.com/healthcare/docs/] > HL7v2IO > FHIRIO > DICOMĀ -- This message was sent by Atlassian Jira (v8.3.4#803005)