You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/08/03 19:56:21 UTC

[GitHub] [beam] pabloem commented on a change in pull request #12331: [BEAM-10601] DICOM API Beam IO connector

pabloem commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r464624146



##########
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##########
@@ -0,0 +1,572 @@
+#
+# 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.
+#
+
+"""DICOM IO connector
+This module implements several tools to facilitate the interaction between
+a Google Cloud Healthcare DICOM store and a Beam pipeline.
+For more details on DICOM store and API:
+https://cloud.google.com/healthcare/docs/how-tos/dicom
+The DICOM IO connector can be used to search metadata or write DICOM files

Review comment:
       extra line as well to separate the paragraphs.
   ```suggestion
   
   The DICOM IO connector can be used to search metadata or write DICOM files
   ```

##########
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##########
@@ -0,0 +1,572 @@
+#
+# 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.
+#
+
+"""DICOM IO connector
+This module implements several tools to facilitate the interaction between
+a Google Cloud Healthcare DICOM store and a Beam pipeline.
+For more details on DICOM store and API:

Review comment:
       add an extra line between 20 and 21? To separate the paragraphs.
   ```suggestion
   
   For more details on DICOM store and API:
   ```

##########
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##########
@@ -0,0 +1,572 @@
+#
+# 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.
+#
+
+"""DICOM IO connector
+This module implements several tools to facilitate the interaction between
+a Google Cloud Healthcare DICOM store and a Beam pipeline.
+For more details on DICOM store and API:
+https://cloud.google.com/healthcare/docs/how-tos/dicom
+The DICOM IO connector can be used to search metadata or write DICOM files
+to DICOM store. When used together with Google Pubsub message connector,
+a PTransform implemented in this module can be used to convert pubsub
+messages to search requests. Since Traceability is crucial for healthcare
+API users, every input or error message will be recorded in the output of
+the DICOM IO connector. As a result, every PTransform in this module will
+return a PCollection of dict that encodes results and detailed error messages.
+
+Search instance's metadata (QIDO request)
+===================================================
+DicomSearch() wraps the QIDO request client and supports 3 levels of search.
+Users should specify the level by setting the 'search_type' entry in the input
+dict. They can also refine the search by adding tags to filter the results using
+the 'params' entry. Here is a sample usage:
+
+  with Pipeline() as p:
+    input_dict = p | beam.Create(
+      [{'project_id': 'abc123', 'type': 'instances',...},
+      {'project_id': 'dicom_go', 'type': 'series',...}])
+
+    results = input_dict | io.gcp.DicomSearch()
+    results | 'print successful search' >> beam.Map(
+    lambda x: print(x['result'] if x['success'] else None))
+
+    results | 'print failed search' >> beam.Map(
+    lambda x: print(x['result'] if not x['success'] else None))
+
+In the example above, successful qido search results and error messages for
+failed requests are printed. When used in real life, user can choose to filter
+those data and output them to wherever they want.
+
+Convert DICOM Pubsub message to Qido search request
+===================================================
+Healthcare API users might use Beam's Pubsub streaming pipeline to monitor the
+store operations (new DICOM file) in a DICOM storage. Pubsub message encodes
+DICOM a web store path as well as instance ids. If users are interested in
+getting new instance's metadata, they can use PubsubToQido() to convert the
+message into Qido Search dict then use DicomSearch(). Here is a sample usage:

Review comment:
       ```suggestion
   Healthcare API users might read messages from Pubsub to monitor the store
   operations (e.g. new file) in a DICOM storage. Pubsub message encode
   DICOM as a web store path as well as instance ids. If users are interested in
   getting new instance's metadata, they can use the `PubsubToQido` transform
   to convert the message into Qido Search dict then use the `DicomSearch`
   transform. Here is a sample usage:
   ```

##########
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##########
@@ -0,0 +1,572 @@
+#
+# 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.
+#
+
+"""DICOM IO connector
+This module implements several tools to facilitate the interaction between
+a Google Cloud Healthcare DICOM store and a Beam pipeline.
+For more details on DICOM store and API:
+https://cloud.google.com/healthcare/docs/how-tos/dicom
+The DICOM IO connector can be used to search metadata or write DICOM files
+to DICOM store. When used together with Google Pubsub message connector,
+a PTransform implemented in this module can be used to convert pubsub
+messages to search requests. Since Traceability is crucial for healthcare
+API users, every input or error message will be recorded in the output of
+the DICOM IO connector. As a result, every PTransform in this module will
+return a PCollection of dict that encodes results and detailed error messages.

Review comment:
       actually I wonder if the transform should be called `FormatToQuido` instead of `PubsubToQuido` is it possible to receive the same data from sources other than pubsub?

##########
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##########
@@ -0,0 +1,572 @@
+#
+# 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.
+#
+
+"""DICOM IO connector
+This module implements several tools to facilitate the interaction between
+a Google Cloud Healthcare DICOM store and a Beam pipeline.
+For more details on DICOM store and API:
+https://cloud.google.com/healthcare/docs/how-tos/dicom
+The DICOM IO connector can be used to search metadata or write DICOM files
+to DICOM store. When used together with Google Pubsub message connector,
+a PTransform implemented in this module can be used to convert pubsub
+messages to search requests. Since Traceability is crucial for healthcare
+API users, every input or error message will be recorded in the output of
+the DICOM IO connector. As a result, every PTransform in this module will
+return a PCollection of dict that encodes results and detailed error messages.

Review comment:
       ```suggestion
   to DICOM store. 
   
   When used together with Google Pubsub message connector, the 
   `PubsubToQuido` PTransform implemented in this module can be used
   to convert Pubsub messages to search requests. 
   
   Since Traceability is crucial for healthcare
   API users, every input or error message will be recorded in the output of
   the DICOM IO connector. As a result, every PTransform in this module will
   return a PCollection of dict that encodes results and detailed error messages.
   ```

##########
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##########
@@ -0,0 +1,572 @@
+#
+# 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.
+#
+
+"""DICOM IO connector
+This module implements several tools to facilitate the interaction between
+a Google Cloud Healthcare DICOM store and a Beam pipeline.
+For more details on DICOM store and API:
+https://cloud.google.com/healthcare/docs/how-tos/dicom
+The DICOM IO connector can be used to search metadata or write DICOM files
+to DICOM store. When used together with Google Pubsub message connector,
+a PTransform implemented in this module can be used to convert pubsub
+messages to search requests. Since Traceability is crucial for healthcare
+API users, every input or error message will be recorded in the output of
+the DICOM IO connector. As a result, every PTransform in this module will
+return a PCollection of dict that encodes results and detailed error messages.
+
+Search instance's metadata (QIDO request)
+===================================================
+DicomSearch() wraps the QIDO request client and supports 3 levels of search.
+Users should specify the level by setting the 'search_type' entry in the input
+dict. They can also refine the search by adding tags to filter the results using
+the 'params' entry. Here is a sample usage:
+
+  with Pipeline() as p:
+    input_dict = p | beam.Create(
+      [{'project_id': 'abc123', 'type': 'instances',...},
+      {'project_id': 'dicom_go', 'type': 'series',...}])
+
+    results = input_dict | io.gcp.DicomSearch()
+    results | 'print successful search' >> beam.Map(
+    lambda x: print(x['result'] if x['success'] else None))
+
+    results | 'print failed search' >> beam.Map(
+    lambda x: print(x['result'] if not x['success'] else None))
+
+In the example above, successful qido search results and error messages for
+failed requests are printed. When used in real life, user can choose to filter
+those data and output them to wherever they want.
+
+Convert DICOM Pubsub message to Qido search request
+===================================================
+Healthcare API users might use Beam's Pubsub streaming pipeline to monitor the
+store operations (new DICOM file) in a DICOM storage. Pubsub message encodes
+DICOM a web store path as well as instance ids. If users are interested in
+getting new instance's metadata, they can use PubsubToQido() to convert the
+message into Qido Search dict then use DicomSearch(). Here is a sample usage:
+
+  pipeline_options = PipelineOptions()
+  pipeline_options.view_as(StandardOptions).streaming = True
+  p =  beam.Pipeline(options=pipeline_options)
+  pubsub = p | beam.io.ReadStringFromPubsub(subscription='a_dicom_store')
+  results = pubsub | PubsubToQido()
+  success = results | 'filter message' >> beam.Filter(lambda x: x['success'])
+  qido_dict = success | 'get qido request' >> beam.Map(lambda x: x['result'])
+  metadata = qido_dict | DicomSearch()
+
+In the example above, the pipeline is listening to a pubsub topic and waiting
+for messages from DICOM API. When a new DICOM file comes into the storage, the
+pipeline will receive a pubsub message, convert it to a Qido request dict and
+feed it to DicomSearch() PTransform. As a result, users can get the metadata for
+every new DICOM file. Note that not every pubsub message received is from DICOM
+API, so we to filter the results first.
+
+Store a DICOM file in a DICOM storage
+===================================================
+DicomStoreInstance() wraps store request API and users can use it to send a
+DICOM file to a DICOM store. It supports two types of input: 1.file data in
+byte[] 2.fileio object. Users should set the 'input_type' when initialzing
+this PTransform. Here are the examples:
+
+  with Pipeline() as p:
+    input_dict = {'project_id': 'abc123', 'type': 'instances',...}
+    path = "gcs://bucketname/something/a.dcm"
+    match = p | fileio.MatchFiles(path)
+    fileio_obj = match | fileio.ReadAll()
+    results = fileio_obj | DicomStoreInstance(input_dict, 'fileio')

Review comment:
       Should this be called `WriteFileToDicomStore`? or perhaps `UploadToDicom` instead of `DicomStoreInstance`?




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