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



##########
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:
       you may need to `return self._flush()`, right?

##########
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:
       again, it may be worth adding a test for this.

##########
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:
       this should be in GCP_REQUIREMENTS, not in REQUIRED_PACKAGES. Why do you 
find you need it here?

##########
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:
       Are you able to add a test for this? a unittest should be fine, to make 
sure that data is returned appropriately?

##########
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:
       as part of a unittest, sometimes it's useful to give users the ability 
to override the DicomApiHttpClient with a mock or mock-like object. This is not 
required, but it may be useful to add unit tests.
   If not, please create a JIRA issue to track this, and add a TODO around here

##########
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:
       and also, it may be good to enable users to override the 
DicomHttpApiClient here

##########
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:
       Was the indentantion causing issues here? It would be good to keep it to 
have some kind of formatting... perhaps you can use dashes to show formatting? 
LMK if we should research a bit together.

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

Review comment:
       curious about the indentation here as well

##########
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:
       you may need to add the timestamp to the buffer as well, just to avoid 
losing the appropriate timestamp of your element (via DoFn.TimestampParam I 
believe)

##########
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:
       you can also be more liberal with the versions. Probably <2 should 
suffice.

##########
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': 'abc123', 'type': 'instances',...}
+    f = open("abc.dcm", "rb")
+    dcm_file = f.read()
+    byte_file = p | 'create byte file' >> beam.Create([dcm_file])
+    results = byte_file | DicomStoreInstance(input_dict, 'bytes')
+
+The first example uses a PCollection of fileio objects as input.
+DicomStoreInstance will read DICOM files from the objects and send them
+to a DICOM storage.
+The second example uses a PCollection of byte[] as input. DicomStoreInstance
+will directly send those DICOM files to a DICOM storage.
+Users can also get the operation results in the output PCollection if they want
+to handle the failed store requests.
+"""
+
+# pytype: skip-file
+from __future__ import absolute_import
+
+import apache_beam as beam
+from apache_beam.io.gcp.dicomclient import DicomApiHttpClient
+from apache_beam.transforms import PTransform
+
+
+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:
+    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),
+    }
+    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)
+      search_type: Which type of search it is, could only be one of the three
+        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"}
+    OUTPUT:
+    The output dict wraps results as well as error messages:
+    {
+      'result': a list of dicts in JSON style.
+      'success': boolean value telling whether the operation is successful.
+      'input': detail ids and dicomweb path for this retrieval.
+      'status': status code from the server, used as error message.
+    }

Review comment:
       this is reasonable. Thanks!

##########
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:
       it may be that your elements don't have a timestamp assigned, but it may 
also be that users will have streaming pipeliness where the timestamps matter

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

Review comment:
       it may be good to rename this class to `_QuidoReadFn`? That's because 
Source is another class type in Beam.

##########
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:
       you have to yield the output from the flush, right?




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


Reply via email to