Class HL7v2IO
- java.lang.Object
-
- org.apache.beam.sdk.io.gcp.healthcare.HL7v2IO
-
public class HL7v2IO extends java.lang.ObjectHL7v2IOprovides an API for reading from and writing to Google Cloud Healthcare HL7v2 API.Read
HL7v2 Messages can be fetched from the HL7v2 store in two ways Message Fetching and Message Listing.
Message Fetching
Message Fetching with
HL7v2IO.Readsupports use cases where you have a $PCollectionof message IDS. This is appropriate for reading the HL7v2 notifications from a Pub/Sub subscription withPubsubIO.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 withTextIO) .Fetch Message contents from HL7v2 Store based on the
PCollectionof message ID stringsHL7v2IO.Read.Resultwhere one can callHL7v2IO.HL7v2ReadResult.getMessages()to retrived aPCollectioncontaining the successfully fetchedHL7v2Messages and/orHL7v2IO.HL7v2ReadResult.getFailedReads()to retrieve aPCollectionofHealthcareIOErrorcontaining the msgID that could not be fetched and the exception as aHealthcareIOError, 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 upstreamPCollectioncontains IDs that are not valid or are not reachable due to permissions issues.Message Listing Message Listing with
HL7v2IO.ListHL7v2Messagessupports batch use cases where you want to process all the messages in an HL7v2 store or those matching a filter- See Also:
- https://cloud.google.com/healthcare/docs/reference/rest/v1/projects.locations.datasets.hl7V2Stores.messages/list#query-parameters
This paginates through results of a Messages.List call @see https://cloud.google.com/healthcare/docs/reference/rest/v1/projects.locations.datasets.hl7V2Stores.messages/list
and outputs directly to a PCollection of 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.
Write
A bounded or unbounded PCollection of HL7v2Message can be ingested into an HL7v2 store using HL7v2IO#ingestMessages(String). This will return a HL7v2IO.Write.Result on which you can call Write.Result#getFailedInsertsWithErr() to retrieve a PCollection of HealthcareIOError containing the HL7v2Message that failed to be ingested and the exception. This can be used to write to the dead letter storage system of your chosing.
Unbounded Read Example:
PipelineOptions options = ...; Pipeline p = Pipeline.create(options); HL7v2IO.Read.Result readResult = p .apply( "Read HL7v2 notifications", PubsubIO.readStrings().fromSubscription(options.getNotificationSubscription())) .apply(HL7v2IO.getAll()); // 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***out = readResult.getMessages().apply("ProcessFetchedMessages", ...); // Write using the Message.Ingest method of the HL7v2 REST API. out.apply(HL7v2IO.ingestMessages(options.getOutputHL7v2Store())); pipeline.run(); Bounded Read Example:
PipelineOptions options = ...; Pipeline p = Pipeline.create(options); PCollection***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();
-
-
Nested Class Summary
Nested Classes Modifier and Type Class Description static classHL7v2IO.HL7v2ReadThe type Read that reads HL7v2 message contents given a PCollection ofHL7v2ReadParameter.static classHL7v2IO.ListHL7v2MessagesList HL7v2 messages in HL7v2 Stores with optional filter.static classHL7v2IO.ReadThe type Read that reads HL7v2 message contents given a PCollection of message IDs strings.static classHL7v2IO.WriteThe type Write that writes the given PCollection of HL7v2 messages.
-
Constructor Summary
Constructors Constructor Description HL7v2IO()
-
Method Summary
All Methods Static Methods Concrete Methods Modifier and Type Method Description static HL7v2IO.ReadgetAll()Retrieve all HL7v2 Messages from a PCollection of message IDs (such as from PubSub notification subscription).static HL7v2IO.WriteingestMessages(java.lang.String hl7v2Store)Write with Messages.Ingest method.static HL7v2IO.ListHL7v2Messagesread(java.lang.String hl7v2Store)Read all HL7v2 Messages from a single store.static HL7v2IO.ListHL7v2Messagesread(org.apache.beam.sdk.options.ValueProvider<java.lang.String> hl7v2Store)Read all HL7v2 Messages from a single store.static HL7v2IO.ListHL7v2MessagesreadAll(java.util.List<java.lang.String> hl7v2Stores)Read all HL7v2 Messages from multiple stores.static HL7v2IO.ListHL7v2MessagesreadAll(org.apache.beam.sdk.options.ValueProvider<java.util.List<java.lang.String>> hl7v2Stores)Read all HL7v2 Messages from multiple stores.static HL7v2IO.HL7v2ReadreadAllRequests()Retrieve all HL7v2 Messages from a PCollection ofHL7v2ReadParameter.static HL7v2IO.ListHL7v2MessagesreadAllWithFilter(java.util.List<java.lang.String> hl7v2Stores, java.lang.String filter)Read all HL7v2 Messages from a multiple stores matching a filter.static HL7v2IO.ListHL7v2MessagesreadAllWithFilter(org.apache.beam.sdk.options.ValueProvider<java.util.List<java.lang.String>> hl7v2Stores, org.apache.beam.sdk.options.ValueProvider<java.lang.String> filter)Read all HL7v2 Messages from a multiple stores matching a filter.static HL7v2IO.ListHL7v2MessagesreadWithFilter(java.lang.String hl7v2Store, java.lang.String filter)Read all HL7v2 Messages from a single store matching a filter.static HL7v2IO.ListHL7v2MessagesreadWithFilter(org.apache.beam.sdk.options.ValueProvider<java.lang.String> hl7v2Store, org.apache.beam.sdk.options.ValueProvider<java.lang.String> filter)Read all HL7v2 Messages from a single store matching a filter.
-
-
-
Method Detail
-
getAll
public static HL7v2IO.Read getAll()
Retrieve all HL7v2 Messages from a PCollection of message IDs (such as from PubSub notification subscription).
-
readAllRequests
public static HL7v2IO.HL7v2Read readAllRequests()
Retrieve all HL7v2 Messages from a PCollection ofHL7v2ReadParameter.
-
readAll
public static HL7v2IO.ListHL7v2Messages readAll(java.util.List<java.lang.String> hl7v2Stores)
Read all HL7v2 Messages from multiple stores.
-
readAll
public static HL7v2IO.ListHL7v2Messages readAll(org.apache.beam.sdk.options.ValueProvider<java.util.List<java.lang.String>> hl7v2Stores)
Read all HL7v2 Messages from multiple stores.
-
read
public static HL7v2IO.ListHL7v2Messages read(java.lang.String hl7v2Store)
Read all HL7v2 Messages from a single store.
-
read
public static HL7v2IO.ListHL7v2Messages read(org.apache.beam.sdk.options.ValueProvider<java.lang.String> hl7v2Store)
Read all HL7v2 Messages from a single store.
-
readWithFilter
public static HL7v2IO.ListHL7v2Messages readWithFilter(java.lang.String hl7v2Store, java.lang.String filter)
Read all HL7v2 Messages from a single store matching a filter.
-
readWithFilter
public static HL7v2IO.ListHL7v2Messages readWithFilter(org.apache.beam.sdk.options.ValueProvider<java.lang.String> hl7v2Store, org.apache.beam.sdk.options.ValueProvider<java.lang.String> filter)
Read all HL7v2 Messages from a single store matching a filter.
-
readAllWithFilter
public static HL7v2IO.ListHL7v2Messages readAllWithFilter(java.util.List<java.lang.String> hl7v2Stores, java.lang.String filter)
Read all HL7v2 Messages from a multiple stores matching a filter.
-
readAllWithFilter
public static HL7v2IO.ListHL7v2Messages readAllWithFilter(org.apache.beam.sdk.options.ValueProvider<java.util.List<java.lang.String>> hl7v2Stores, org.apache.beam.sdk.options.ValueProvider<java.lang.String> filter)
Read all HL7v2 Messages from a multiple stores matching a filter.
-
ingestMessages
public static HL7v2IO.Write ingestMessages(java.lang.String hl7v2Store)
- Parameters:
hl7v2Store- the hl 7 v 2 store- Returns:
- the write
-
-