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

2020-08-03 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r464636772



##
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:
   changed





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

2020-08-03 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r464632452



##
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:
   You are right, although most of the time it's used together with pubsub, 
the string can come from any source.





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




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

2020-08-03 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r464632452



##
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:
   You are right, although most of the time it's used together with pubsub, 
but the string can come from any source.





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




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

2020-07-31 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r463813581



##
File path: sdks/python/setup.py
##
@@ -128,6 +128,7 @@ def get_version():
   cythonize = lambda *args, **kwargs: []
 
 REQUIRED_PACKAGES = [
+'google-auth>=1.18.0,<=1.20.0',

Review comment:
   Fixed





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




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

2020-07-31 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r463383155



##
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##
@@ -164,70 +167,109 @@ class DicomSearch(PTransform):
 }
 
   """
-  def __init__(self, credential=None):
+  def __init__(self, buffer_size=8, max_workers=5, credential=None):
 """Initializes DicomSearch.
 Args:
   credential: # type: Google credential object, if it is specified, the
   Http client will use it to create sessions instead of the default.
 """
+self.buffer_size = buffer_size
+self.max_workers = max_workers

Review comment:
   Added, but for some objects reference issues in the unit test, it has to 
be added in an ugly way in the WriteToDicomStore connector...





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




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

2020-07-30 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r463411782



##
File path: sdks/python/setup.py
##
@@ -128,6 +128,7 @@ def get_version():
   cythonize = lambda *args, **kwargs: []
 
 REQUIRED_PACKAGES = [
+'google-auth>=1.18.0,<=1.20.0',

Review comment:
   For some reason if I put it under GCP_REQUIREMENTS, there will still be 
'no mudule found
   errors caused by google.auth during the unittest. 
https://ci-beam.apache.org/job/beam_PreCommit_Python_Commit/14212/testReport/junit/(root)/(empty)/apache_beam_io_gcp_dicomio_test_7/

##
File path: sdks/python/setup.py
##
@@ -128,6 +128,7 @@ def get_version():
   cythonize = lambda *args, **kwargs: []
 
 REQUIRED_PACKAGES = [
+'google-auth>=1.18.0,<=1.20.0',

Review comment:
   For some reason if I put it under GCP_REQUIREMENTS, there will be 'no 
mudule found
   errors caused by google.auth during the unittest. 
https://ci-beam.apache.org/job/beam_PreCommit_Python_Commit/14212/testReport/junit/(root)/(empty)/apache_beam_io_gcp_dicomio_test_7/





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




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

2020-07-30 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r463411782



##
File path: sdks/python/setup.py
##
@@ -128,6 +128,7 @@ def get_version():
   cythonize = lambda *args, **kwargs: []
 
 REQUIRED_PACKAGES = [
+'google-auth>=1.18.0,<=1.20.0',

Review comment:
   For some reason if I put it under GCP_REQUIREMENTS, there will be 
'ModuleNotFoundError' caused by google.auth during the unittest. 
https://ci-beam.apache.org/job/beam_PreCommit_Python_Commit/14212/testReport/junit/(root)/(empty)/apache_beam_io_gcp_dicomio_test_7/





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




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

2020-07-30 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r463383155



##
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##
@@ -164,70 +167,109 @@ class DicomSearch(PTransform):
 }
 
   """
-  def __init__(self, credential=None):
+  def __init__(self, buffer_size=8, max_workers=5, credential=None):
 """Initializes DicomSearch.
 Args:
   credential: # type: Google credential object, if it is specified, the
   Http client will use it to create sessions instead of the default.
 """
+self.buffer_size = buffer_size
+self.max_workers = max_workers

Review comment:
   Added, but for some objects reference issues in the unit test it has to 
be added in an ugly way in the WriteToDicomStore connector...





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




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

2020-07-30 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r463382756



##
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##
@@ -372,52 +420,63 @@ def __init__(self, destination_dict, input_type, 
credential=None):
   credential: # type: Google credential object, if it is specified, the
   Http client will use it instead of the default one.
 """
-self.credential = credential
 self.destination_dict = destination_dict
 # input_type pre-check
 if input_type not in ['bytes', 'fileio']:
   raise ValueError("input_type could only be 'bytes' or 'fileio'")
 self.input_type = input_type
+self.buffer_size = buffer_size
+self.max_workers = max_workers
+self.credential = credential
 
   def expand(self, pcoll):
 return pcoll | beam.ParDo(
-_StoreInstance(self.destination_dict, self.input_type, 
self.credential))
+_StoreInstance(
+self.destination_dict,
+self.input_type,
+self.buffer_size,
+self.max_workers,
+self.credential))
 
 
 class _StoreInstance(beam.DoFn):
   """A DoFn read or fetch dicom files then push it to a dicom store."""
-  def __init__(self, destination_dict, input_type, credential=None):
-self.credential = credential
+  def __init__(
+  self,
+  destination_dict,
+  input_type,
+  buffer_size,
+  max_workers,
+  credential=None):

Review comment:
   Add custom client support

##
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##
@@ -164,70 +167,109 @@ class DicomSearch(PTransform):
 }
 
   """
-  def __init__(self, credential=None):
+  def __init__(self, buffer_size=8, max_workers=5, credential=None):
 """Initializes DicomSearch.
 Args:
   credential: # type: Google credential object, if it is specified, the
   Http client will use it to create sessions instead of the default.
 """
+self.buffer_size = buffer_size
+self.max_workers = max_workers
 self.credential = credential
 
   def expand(self, pcoll):
-return pcoll | beam.ParDo(_QidoSource(self.credential))
+return pcoll | beam.ParDo(
+_QidoSource(self.buffer_size, self.max_workers, self.credential))
 
 
 class _QidoSource(beam.DoFn):
   """A DoFn for executing every qido query request."""
-  def __init__(self, credential=None):
+  def __init__(self, buffer_size, max_workers, credential=None):
+self.buffer_size = buffer_size
+self.max_workers = max_workers
 self.credential = credential
 
-  def process(self, element):
+  def start_bundle(self):
+self.buffer = []
+
+  def finish_bundle(self):
+return self._flush()
+
+  def validate_element(self, element):
 # Check if all required keys present.
 required_keys = [
 'project_id', 'region', 'dataset_id', 'dicom_store_id', 'search_type'
 ]
 
-error_message = None
-
 for key in required_keys:
   if key not in element:
 error_message = 'Must have %s in the dict.' % (key)
-break
-
-if not error_message:
-  project_id = element['project_id']
-  region = element['region']
-  dataset_id = element['dataset_id']
-  dicom_store_id = element['dicom_store_id']
-  search_type = element['search_type']
-  params = element['params'] if 'params' in element else None
-
-  # Call qido search http client
-  if element['search_type'] in ['instances', "studies", "series"]:
-result, status_code = DicomApiHttpClient().qido_search(
-  project_id, region, dataset_id, dicom_store_id,
-  search_type, params, self.credential
-)
-  else:
-error_message = (
-'Search type can only be "studies", '
-'"instances" or "series"')
-
-  if not error_message:
-out = {}
-out['result'] = result
-out['status'] = status_code
-out['input'] = element
-out['success'] = (status_code == 200)
-return [out]
-
-# Return this when the input dict dose not meet the requirements
+return False, error_message
+
+# Check if return type is correct.
+if element['search_type'] in ['instances', "studies", "series"]:
+  return True, None
+else:
+  error_message = (
+  'Search type can only be "studies", '
+  '"instances" or "series"')
+  return False, error_message
+
+  def process(self, element, window=beam.DoFn.WindowParam):
+# Check if the element is valid
+valid, error_message = self.validate_element(element)
+
+if valid:
+  self.buffer.append((element, window))
+  if len(self.buffer) >= self.buffer_size:
+self._flush()

Review comment:
   Fixed and test added

##
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##
@@ -164,70 +167,109 @@ class DicomSearch(PTransform):
 }
 
   """
-  def __init__(self, credential=None):
+  def 

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

2020-07-30 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r463382633



##
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##
@@ -372,52 +420,63 @@ def __init__(self, destination_dict, input_type, 
credential=None):
   credential: # type: Google credential object, if it is specified, the
   Http client will use it instead of the default one.
 """
-self.credential = credential
 self.destination_dict = destination_dict
 # input_type pre-check
 if input_type not in ['bytes', 'fileio']:
   raise ValueError("input_type could only be 'bytes' or 'fileio'")
 self.input_type = input_type
+self.buffer_size = buffer_size
+self.max_workers = max_workers
+self.credential = credential
 
   def expand(self, pcoll):
 return pcoll | beam.ParDo(
-_StoreInstance(self.destination_dict, self.input_type, 
self.credential))
+_StoreInstance(
+self.destination_dict,
+self.input_type,
+self.buffer_size,
+self.max_workers,
+self.credential))
 
 
 class _StoreInstance(beam.DoFn):
   """A DoFn read or fetch dicom files then push it to a dicom store."""
-  def __init__(self, destination_dict, input_type, credential=None):
-self.credential = credential
+  def __init__(
+  self,
+  destination_dict,
+  input_type,
+  buffer_size,
+  max_workers,
+  credential=None):
 # pre-check destination dict
 required_keys = ['project_id', 'region', 'dataset_id', 'dicom_store_id']
 for key in required_keys:
   if key not in destination_dict:
 raise ValueError('Must have %s in the dict.' % (key))
 self.destination_dict = destination_dict
 self.input_type = input_type
+self.buffer_size = buffer_size
+self.max_workers = max_workers
+self.credential = credential
 
-  def process(self, element):
+  def start_bundle(self):
+self.buffer = []
+
+  def finish_bundle(self):
+return self._flush()
+
+  def process(self, element, window=beam.DoFn.WindowParam):
+self.buffer.append((element, window))
+if len(self.buffer) >= self.buffer_size:
+  self._flush()

Review comment:
   Good catch, fixed and tests added!





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




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

2020-07-30 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r463382681



##
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##
@@ -426,6 +485,44 @@ def process(self, element):
 
 out = {}
 out['status'] = status_code
-out['input'] = None if status_code == 200 else element
+out['input'] = None if status_code == 200 else dicom_file
 out['success'] = (status_code == 200)
-return [out]
+return out
+
+  def read_dicom_file(self, buffer_element):
+# Read the file based on different input. If the read fails ,return
+# an error dict which records input and error messages.
+try:
+  if self.input_type == 'fileio':
+f = buffer_element.open()
+return True, f.read()
+  else:
+return True, buffer_element
+except Exception as error_message:
+  error_out = {}
+  error_out['status'] = error_message
+  error_out['input'] = buffer_element
+  error_out['success'] = False
+  return False, error_out
+
+  def process_buffer_element(self, buffer_element):
+# Thread job runner - each thread stores a DICOM file
+success, read_result = self.read_dicom_file(buffer_element[0])
+windows = [buffer_element[1]]
+value = None
+if success:
+  value = self.make_request(read_result)
+else:
+  value = read_result
+return beam.utils.windowed_value.WindowedValue(
+value=value, timestamp=0, windows=windows)

Review comment:
   Added timestamps





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




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

2020-07-30 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r463382501



##
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##
@@ -114,55 +115,60 @@
 
 class DicomSearch(PTransform):
   """A PTransform used for retrieving DICOM instance metadata from Google
-Cloud DICOM store. It takes a Pcollection of dicts as input and return
-a Pcollection of dict as results:
+Cloud DICOM store. It takes a PCollection of dicts as input and return
+a PCollection of dict as results:
 INPUT:
 The input dict represents DICOM web path parameters, which has the 
following
 string keys and values:
 {
-  'project_id': str,
-  'region': str,
-  'dataset_id': str,
-  'dicom_store_id': str,
-  'search_type': str,
-  'params': dict(str,str) (Optional),
+'project_id': str,
+'region': str,
+'dataset_id': str,
+'dicom_store_id': str,
+'search_type': str,
+'params': dict(str,str) (Optional),
 }
+
 Key-value pairs:
-  project_id: Id of the project in which DICOM store locates. (Required)
+  project_id: Id of the project in which the DICOM store is
+  located. (Required)
   region: Region where the DICOM store resides. (Required)
   dataset_id: Id of the dataset where DICOM store belongs to. (Required)
   dicom_store_id: Id of the dicom store. (Required)
   search_type: Which type of search it is, could only be one of the three
-values: 'instances', 'series', or 'studies'. (Required)
+  values: 'instances', 'series', or 'studies'. (Required)
   params: A dict of str:str pairs used to refine QIDO search. (Optional)
-Supported tags in three categories:
-  1. Studies:
-StudyInstanceUID
-PatientName
-PatientID
-AccessionNumber
-ReferringPhysicianName
-StudyDate
-  2. Series: all study level search terms and
-SeriesInstanceUID
-Modality
-  3. Instances: all study/series level search terms and
-SOPInstanceUID
-e.g. {"StudyInstanceUID":"1","SeriesInstanceUID":"2"}
+  Supported tags in three categories:
+  1.Studies:
+  StudyInstanceUID,
+  PatientName,
+  PatientID,
+  AccessionNumber,
+  ReferringPhysicianName,
+  StudyDate,
+  2.Series: all study level search terms and
+  SeriesInstanceUID,
+  Modality,
+  3.Instances: all study/series level search terms and
+  SOPInstanceUID,

Review comment:
   Fixed

##
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##
@@ -324,41 +332,45 @@ def process(self, element):
 return [out]
 
 
-class DicomStoreInstance(PTransform):
+class WriteToDicomStore(PTransform):
   """A PTransform for storing instances to a DICOM store.
-Takes Pcollection of byte[] as input and return a Pcollection of dict as
+Takes PCollection of byte[] as input and return a PCollection of dict as
 results. The inputs are normally DICOM file in bytes or str filename.
 INPUT:
-  This PTransform supports two types of input:
-1. Byte[]: representing dicom file.
-2. Fileio object: stream file object.
+This PTransform supports two types of input:
+1. Byte[]: representing dicom file.
+2. Fileio object: stream file object.
+
 OUTPUT:
 The output dict encodes status as well as error messages:
 {
-  'success': boolean value telling whether the store is successful
-  'input': undeliverable data. Exactly the same as the input,
-only set if the operation is failed.
-  'status': status code from the server, used as error messages.
+'success': boolean value telling whether the store is successful.
+'input': undeliverable data. Exactly the same as the input,
+only set if the operation is failed.
+'status': status code from the server, used as error messages.
 }
+
   """
   def __init__(self, destination_dict, input_type, credential=None):
-"""Initializes DicomStoreInstance.
+"""Initializes WriteToDicomStore.
 Args:
   destination_dict: # type: python dict, encodes DICOM endpoint 
information:
-{
-  'project_id': str,
-  'region': str,
-  'dataset_id': str,
-  'dicom_store_id': str,
-}
-Key-value pairs:
-  project_id: Id of the project in which DICOM store locates. 
(Required)
-  region: Region where the DICOM store resides. (Required)
-  dataset_id: Id of the dataset where DICOM store belongs to. 
(Required)
-  dicom_store_id: Id of the dicom store. (Required)
+  {
+  'project_id': str,
+  'region': str,
+  'dataset_id': str,
+  'dicom_store_id': str,
+  }
+
+  Key-value pairs:
+  project_id: Id of the project in which DICOM store locates. (Required)
+  region: Region where the 

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

2020-07-30 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r463019759



##
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##
@@ -0,0 +1,419 @@
+#
+# 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 serval 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
+DICOM io connector can be used to search metadata or store DICOM files.
+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
+  with beam.Pipeline(options=pipeline_options) as p:
+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')
+
+  with Pipeline() as p:
+input_dict = {'project_id': 

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

2020-07-30 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r463003111



##
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##
@@ -0,0 +1,419 @@
+#
+# 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 serval 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
+DICOM io connector can be used to search metadata or store DICOM files.
+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
+  with beam.Pipeline(options=pipeline_options) as p:
+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')
+
+  with Pipeline() as p:
+input_dict = {'project_id': 

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

2020-07-30 Thread GitBox


George-Wu commented on a change in pull request #12331:
URL: https://github.com/apache/beam/pull/12331#discussion_r463003111



##
File path: sdks/python/apache_beam/io/gcp/dicomio.py
##
@@ -0,0 +1,419 @@
+#
+# 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 serval 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
+DICOM io connector can be used to search metadata or store DICOM files.
+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
+  with beam.Pipeline(options=pipeline_options) as p:
+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')
+
+  with Pipeline() as p:
+input_dict = {'project_id':