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

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

                Author: ASF GitHub Bot
            Created on: 18/Mar/20 22:52
            Start Date: 18/Mar/20 22:52
    Worklog Time Spent: 10m 
      Work Description: pabloem commented on 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:
us...@infra.apache.org


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

    Worklog Id:     (was: 405808)
    Time Spent: 2h 40m  (was: 2.5h)

> 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: 2h 40m
>  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)

Reply via email to