github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/io/gcp/dicomio.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 # 17 18 """DICOM IO connector 19 This module implements several tools to facilitate the interaction between 20 a Google Cloud Healthcare DICOM store and a Beam pipeline. 21 22 For more details on DICOM store and API: 23 https://cloud.google.com/healthcare/docs/how-tos/dicom 24 25 The DICOM IO connector can be used to search metadata or write DICOM files 26 to DICOM store. 27 28 When used together with Google Pubsub message connector, the 29 `FormatToQido` PTransform implemented in this module can be used 30 to convert Pubsub messages to search requests. 31 32 Since Traceability is crucial for healthcare 33 API users, every input or error message will be recorded in the output of 34 the DICOM IO connector. As a result, every PTransform in this module will 35 return a PCollection of dict that encodes results and detailed error messages. 36 37 Search instance's metadata (QIDO request) 38 =================================================== 39 DicomSearch() wraps the QIDO request client and supports 3 levels of search. 40 Users should specify the level by setting the 'search_type' entry in the input 41 dict. They can also refine the search by adding tags to filter the results using 42 the 'params' entry. Here is a sample usage: 43 44 with Pipeline() as p: 45 input_dict = p | beam.Create( 46 [{'project_id': 'abc123', 'type': 'instances',...}, 47 {'project_id': 'dicom_go', 'type': 'series',...}]) 48 49 results = input_dict | io.gcp.DicomSearch() 50 results | 'print successful search' >> beam.Map( 51 lambda x: print(x['result'] if x['success'] else None)) 52 53 results | 'print failed search' >> beam.Map( 54 lambda x: print(x['result'] if not x['success'] else None)) 55 56 In the example above, successful qido search results and error messages for 57 failed requests are printed. When used in real life, user can choose to filter 58 those data and output them to wherever they want. 59 60 Convert DICOM Pubsub message to Qido search request 61 =================================================== 62 Healthcare API users might read messages from Pubsub to monitor the store 63 operations (e.g. new file) in a DICOM storage. Pubsub message encode 64 DICOM as a web store path as well as instance ids. If users are interested in 65 getting new instance's metadata, they can use the `FormatToQido` transform 66 to convert the message into Qido Search dict then use the `DicomSearch` 67 transform. Here is a sample usage: 68 69 pipeline_options = PipelineOptions() 70 pipeline_options.view_as(StandardOptions).streaming = True 71 p = beam.Pipeline(options=pipeline_options) 72 pubsub = p | beam.io.ReadStringFromPubsub(subscription='a_dicom_store') 73 results = pubsub | FormatToQido() 74 success = results | 'filter message' >> beam.Filter(lambda x: x['success']) 75 qido_dict = success | 'get qido request' >> beam.Map(lambda x: x['result']) 76 metadata = qido_dict | DicomSearch() 77 78 In the example above, the pipeline is listening to a pubsub topic and waiting 79 for messages from DICOM API. When a new DICOM file comes into the storage, the 80 pipeline will receive a pubsub message, convert it to a Qido request dict and 81 feed it to DicomSearch() PTransform. As a result, users can get the metadata for 82 every new DICOM file. Note that not every pubsub message received is from DICOM 83 API, so we to filter the results first. 84 85 Store a DICOM file in a DICOM storage 86 =================================================== 87 UploadToDicomStore() wraps store request API and users can use it to send a 88 DICOM file to a DICOM store. It supports two types of input: 1.file data in 89 byte[] 2.fileio object. Users should set the 'input_type' when initialzing 90 this PTransform. Here are the examples: 91 92 with Pipeline() as p: 93 input_dict = {'project_id': 'abc123', 'type': 'instances',...} 94 path = "gcs://bucketname/something/a.dcm" 95 match = p | fileio.MatchFiles(path) 96 fileio_obj = match | fileio.ReadAll() 97 results = fileio_obj | UploadToDicomStore(input_dict, 'fileio') 98 99 with Pipeline() as p: 100 input_dict = {'project_id': 'abc123', 'type': 'instances',...} 101 f = open("abc.dcm", "rb") 102 dcm_file = f.read() 103 byte_file = p | 'create byte file' >> beam.Create([dcm_file]) 104 results = byte_file | UploadToDicomStore(input_dict, 'bytes') 105 106 The first example uses a PCollection of fileio objects as input. 107 UploadToDicomStore will read DICOM files from the objects and send them 108 to a DICOM storage. 109 The second example uses a PCollection of byte[] as input. UploadToDicomStore 110 will directly send those DICOM files to a DICOM storage. 111 Users can also get the operation results in the output PCollection if they want 112 to handle the failed store requests. 113 """ 114 115 # pytype: skip-file 116 from concurrent.futures import ThreadPoolExecutor 117 from concurrent.futures import as_completed 118 119 import apache_beam as beam 120 from apache_beam.io.gcp.dicomclient import DicomApiHttpClient 121 from apache_beam.transforms import PTransform 122 123 124 class DicomSearch(PTransform): 125 """A PTransform used for retrieving DICOM instance metadata from Google 126 Cloud DICOM store. It takes a PCollection of dicts as input and return 127 a PCollection of dict as results: 128 INPUT: 129 The input dict represents DICOM web path parameters, which has the following 130 string keys and values: 131 { 132 'project_id': str, 133 'region': str, 134 'dataset_id': str, 135 'dicom_store_id': str, 136 'search_type': str, 137 'params': dict(str,str) (Optional), 138 } 139 140 Key-value pairs: 141 project_id: Id of the project in which the DICOM store is 142 located. (Required) 143 region: Region where the DICOM store resides. (Required) 144 dataset_id: Id of the dataset where DICOM store belongs to. (Required) 145 dicom_store_id: Id of the dicom store. (Required) 146 search_type: Which type of search it is, could only be one of the three 147 values: 'instances', 'series', or 'studies'. (Required) 148 params: A dict of str:str pairs used to refine QIDO search. (Optional) 149 Supported tags in three categories: 150 1.Studies: 151 * StudyInstanceUID, 152 * PatientName, 153 * PatientID, 154 * AccessionNumber, 155 * ReferringPhysicianName, 156 * StudyDate, 157 2.Series: all study level search terms and 158 * SeriesInstanceUID, 159 * Modality, 160 3.Instances: all study/series level search terms and 161 * SOPInstanceUID, 162 163 e.g. {"StudyInstanceUID":"1","SeriesInstanceUID":"2"} 164 165 OUTPUT: 166 The output dict wraps results as well as error messages: 167 { 168 'result': a list of dicts in JSON style. 169 'success': boolean value telling whether the operation is successful. 170 'input': detail ids and dicomweb path for this retrieval. 171 'status': status code from the server, used as error message. 172 } 173 174 """ 175 def __init__( 176 self, buffer_size=8, max_workers=5, client=None, credential=None): 177 """Initializes DicomSearch. 178 Args: 179 buffer_size: # type: Int. Size of the request buffer. 180 max_workers: # type: Int. Maximum number of threads a worker can 181 create. If it is set to one, all the request will be processed 182 sequentially in a worker. 183 client: # type: object. If it is specified, all the Api calls will 184 made by this client instead of the default one (DicomApiHttpClient). 185 credential: # type: Google credential object, if it is specified, the 186 Http client will use it to create sessions instead of the default. 187 """ 188 self.buffer_size = buffer_size 189 self.max_workers = max_workers 190 self.client = client or DicomApiHttpClient() 191 self.credential = credential 192 193 def expand(self, pcoll): 194 return pcoll | beam.ParDo( 195 _QidoReadFn( 196 self.buffer_size, self.max_workers, self.client, self.credential)) 197 198 199 class _QidoReadFn(beam.DoFn): 200 """A DoFn for executing every qido query request.""" 201 def __init__(self, buffer_size, max_workers, client, credential=None): 202 self.buffer_size = buffer_size 203 self.max_workers = max_workers 204 self.client = client 205 self.credential = credential 206 207 def start_bundle(self): 208 self.buffer = [] 209 210 def finish_bundle(self): 211 for item in self._flush(): 212 yield item 213 214 def validate_element(self, element): 215 # Check if all required keys present. 216 required_keys = [ 217 'project_id', 'region', 'dataset_id', 'dicom_store_id', 'search_type' 218 ] 219 220 for key in required_keys: 221 if key not in element: 222 error_message = 'Must have %s in the dict.' % (key) 223 return False, error_message 224 225 # Check if return type is correct. 226 if element['search_type'] in ['instances', "studies", "series"]: 227 return True, None 228 else: 229 error_message = ( 230 'Search type can only be "studies", ' 231 '"instances" or "series"') 232 return False, error_message 233 234 def process( 235 self, 236 element, 237 window=beam.DoFn.WindowParam, 238 timestamp=beam.DoFn.TimestampParam): 239 # Check if the element is valid 240 valid, error_message = self.validate_element(element) 241 242 if valid: 243 self.buffer.append((element, window, timestamp)) 244 if len(self.buffer) >= self.buffer_size: 245 for item in self._flush(): 246 yield item 247 else: 248 # Return this when the input dict dose not meet the requirements 249 out = {} 250 out['result'] = [] 251 out['status'] = error_message 252 out['input'] = element 253 out['success'] = False 254 yield out 255 256 def make_request(self, element): 257 # Sending Qido request to DICOM Api 258 project_id = element['project_id'] 259 region = element['region'] 260 dataset_id = element['dataset_id'] 261 dicom_store_id = element['dicom_store_id'] 262 search_type = element['search_type'] 263 params = element['params'] if 'params' in element else None 264 265 # Call qido search http client 266 result, status_code = self.client.qido_search( 267 project_id, region, dataset_id, dicom_store_id, 268 search_type, params, self.credential 269 ) 270 271 out = {} 272 out['result'] = result 273 out['status'] = status_code 274 out['input'] = element 275 out['success'] = (status_code == 200) 276 return out 277 278 def process_buffer_element(self, buffer_element): 279 # Thread job runner - each thread makes a Qido search request 280 value = self.make_request(buffer_element[0]) 281 windows = [buffer_element[1]] 282 timestamp = buffer_element[2] 283 return beam.utils.windowed_value.WindowedValue( 284 value=value, timestamp=timestamp, windows=windows) 285 286 def _flush(self): 287 # Create thread pool executor and process the buffered elements in paralllel 288 executor = ThreadPoolExecutor(max_workers=self.max_workers) 289 futures = [ 290 executor.submit(self.process_buffer_element, ele) for ele in self.buffer 291 ] 292 self.buffer = [] 293 for f in as_completed(futures): 294 yield f.result() 295 296 297 class FormatToQido(PTransform): 298 """A PTransform for converting pubsub messages into search input dict. 299 Takes PCollection of string as input and returns a PCollection of dict as 300 results. Note that some pubsub messages may not be from DICOM API, which 301 will be recorded as failed conversions. 302 INPUT: 303 The input are normally strings from Pubsub topic: 304 "projects/PROJECT_ID/locations/LOCATION/datasets/DATASET_ID/ 305 dicomStores/DICOM_STORE_ID/dicomWeb/studies/STUDY_UID/ 306 series/SERIES_UID/instances/INSTANCE_UID" 307 308 OUTPUT: 309 The output dict encodes results as well as error messages: 310 { 311 'result': a dict representing instance level qido search request. 312 'success': boolean value telling whether the conversion is successful. 313 'input': input pubsub message string. 314 } 315 316 """ 317 def __init__(self, credential=None): 318 """Initializes FormatToQido. 319 Args: 320 credential: # type: Google credential object, if it is specified, the 321 Http client will use it instead of the default one. 322 """ 323 self.credential = credential 324 325 def expand(self, pcoll): 326 return pcoll | beam.ParDo(_ConvertStringToQido()) 327 328 329 class _ConvertStringToQido(beam.DoFn): 330 """A DoFn for converting pubsub string to qido search parameters.""" 331 def process(self, element): 332 # Some constants for DICOM pubsub message 333 NUM_PUBSUB_STR_ENTRIES = 15 334 NUM_DICOM_WEBPATH_PARAMETERS = 5 335 NUM_TOTAL_PARAMETERS = 8 336 INDEX_PROJECT_ID = 1 337 INDEX_REGION = 3 338 INDEX_DATASET_ID = 5 339 INDEX_DICOMSTORE_ID = 7 340 INDEX_STUDY_ID = 10 341 INDEX_SERIE_ID = 12 342 INDEX_INSTANCE_ID = 14 343 344 entries = element.split('/') 345 346 # Output dict with error message, used when 347 # receiving invalid pubsub string. 348 error_dict = {} 349 error_dict['result'] = {} 350 error_dict['input'] = element 351 error_dict['success'] = False 352 353 if len(entries) != NUM_PUBSUB_STR_ENTRIES: 354 return [error_dict] 355 356 required_keys = [ 357 'projects', 358 'locations', 359 'datasets', 360 'dicomStores', 361 'dicomWeb', 362 'studies', 363 'series', 364 'instances' 365 ] 366 367 # Check if the required keys present and 368 # the positions of those keys are correct 369 for i in range(NUM_DICOM_WEBPATH_PARAMETERS): 370 if required_keys[i] != entries[i * 2]: 371 return [error_dict] 372 for i in range(NUM_DICOM_WEBPATH_PARAMETERS, NUM_TOTAL_PARAMETERS): 373 if required_keys[i] != entries[i * 2 - 1]: 374 return [error_dict] 375 376 # Compose dicom webpath parameters for qido search 377 qido_dict = {} 378 qido_dict['project_id'] = entries[INDEX_PROJECT_ID] 379 qido_dict['region'] = entries[INDEX_REGION] 380 qido_dict['dataset_id'] = entries[INDEX_DATASET_ID] 381 qido_dict['dicom_store_id'] = entries[INDEX_DICOMSTORE_ID] 382 qido_dict['search_type'] = 'instances' 383 384 # Compose instance level params for qido search 385 params = {} 386 params['StudyInstanceUID'] = entries[INDEX_STUDY_ID] 387 params['SeriesInstanceUID'] = entries[INDEX_SERIE_ID] 388 params['SOPInstanceUID'] = entries[INDEX_INSTANCE_ID] 389 qido_dict['params'] = params 390 391 out = {} 392 out['result'] = qido_dict 393 out['input'] = element 394 out['success'] = True 395 396 return [out] 397 398 399 class UploadToDicomStore(PTransform): 400 """A PTransform for storing instances to a DICOM store. 401 Takes PCollection of byte[] as input and return a PCollection of dict as 402 results. The inputs are normally DICOM file in bytes or str filename. 403 INPUT: 404 This PTransform supports two types of input: 405 1. Byte[]: representing dicom file. 406 2. Fileio object: stream file object. 407 408 OUTPUT: 409 The output dict encodes status as well as error messages: 410 { 411 'success': boolean value telling whether the store is successful. 412 'input': undeliverable data. Exactly the same as the input, 413 only set if the operation is failed. 414 'status': status code from the server, used as error messages. 415 } 416 417 """ 418 def __init__( 419 self, 420 destination_dict, 421 input_type, 422 buffer_size=8, 423 max_workers=5, 424 client=None, 425 credential=None): 426 """Initializes UploadToDicomStore. 427 Args: 428 destination_dict: # type: python dict, encodes DICOM endpoint information: 429 { 430 'project_id': str, 431 'region': str, 432 'dataset_id': str, 433 'dicom_store_id': str, 434 } 435 436 Key-value pairs: 437 * project_id: Id of the project in which DICOM store locates. (Required) 438 * region: Region where the DICOM store resides. (Required) 439 * dataset_id: Id of the dataset where DICOM store belongs to. (Required) 440 * dicom_store_id: Id of the dicom store. (Required) 441 442 input_type: # type: string, could only be 'bytes' or 'fileio' 443 buffer_size: # type: Int. Size of the request buffer. 444 max_workers: # type: Int. Maximum number of threads a worker can 445 create. If it is set to one, all the request will be processed 446 sequentially in a worker. 447 client: # type: object. If it is specified, all the Api calls will 448 made by this client instead of the default one (DicomApiHttpClient). 449 credential: # type: Google credential object, if it is specified, the 450 Http client will use it instead of the default one. 451 """ 452 self.destination_dict = destination_dict 453 # input_type pre-check 454 if input_type not in ['bytes', 'fileio']: 455 raise ValueError("input_type could only be 'bytes' or 'fileio'") 456 self.input_type = input_type 457 self.buffer_size = buffer_size 458 self.max_workers = max_workers 459 self.client = client 460 self.credential = credential 461 462 def expand(self, pcoll): 463 return pcoll | beam.ParDo( 464 _StoreInstance( 465 self.destination_dict, 466 self.input_type, 467 self.buffer_size, 468 self.max_workers, 469 self.client, 470 self.credential)) 471 472 473 class _StoreInstance(beam.DoFn): 474 """A DoFn read or fetch dicom files then push it to a dicom store.""" 475 def __init__( 476 self, 477 destination_dict, 478 input_type, 479 buffer_size, 480 max_workers, 481 client, 482 credential=None): 483 # pre-check destination dict 484 required_keys = ['project_id', 'region', 'dataset_id', 'dicom_store_id'] 485 for key in required_keys: 486 if key not in destination_dict: 487 raise ValueError('Must have %s in the dict.' % (key)) 488 self.destination_dict = destination_dict 489 self.input_type = input_type 490 self.buffer_size = buffer_size 491 self.max_workers = max_workers 492 self.client = client 493 self.credential = credential 494 495 def start_bundle(self): 496 self.buffer = [] 497 498 def finish_bundle(self): 499 for item in self._flush(): 500 yield item 501 502 def process( 503 self, 504 element, 505 window=beam.DoFn.WindowParam, 506 timestamp=beam.DoFn.TimestampParam): 507 self.buffer.append((element, window, timestamp)) 508 if len(self.buffer) >= self.buffer_size: 509 for item in self._flush(): 510 yield item 511 512 def make_request(self, dicom_file): 513 # Send file to DICOM store and records the results. 514 project_id = self.destination_dict['project_id'] 515 region = self.destination_dict['region'] 516 dataset_id = self.destination_dict['dataset_id'] 517 dicom_store_id = self.destination_dict['dicom_store_id'] 518 519 # Feed the dicom file into store client 520 if self.client: 521 _, status_code = self.client.dicomweb_store_instance( 522 project_id, region, dataset_id, dicom_store_id, dicom_file, 523 self.credential 524 ) 525 else: 526 _, status_code = DicomApiHttpClient().dicomweb_store_instance( 527 project_id, region, dataset_id, dicom_store_id, dicom_file, 528 self.credential 529 ) 530 531 out = {} 532 out['status'] = status_code 533 out['success'] = (status_code == 200) 534 return out 535 536 def read_dicom_file(self, buffer_element): 537 # Read the file based on different input. If the read fails ,return 538 # an error dict which records input and error messages. 539 try: 540 if self.input_type == 'fileio': 541 f = buffer_element.open() 542 data = f.read() 543 f.close() 544 return True, data 545 else: 546 return True, buffer_element 547 except Exception as error_message: 548 error_out = {} 549 error_out['status'] = error_message 550 error_out['success'] = False 551 return False, error_out 552 553 def process_buffer_element(self, buffer_element): 554 # Thread job runner - each thread stores a DICOM file 555 success, read_result = self.read_dicom_file(buffer_element[0]) 556 windows = [buffer_element[1]] 557 timestamp = buffer_element[2] 558 value = None 559 if success: 560 value = self.make_request(read_result) 561 else: 562 value = read_result 563 # save the undeliverable data 564 if not value['success']: 565 value['input'] = buffer_element[0] 566 return beam.utils.windowed_value.WindowedValue( 567 value=value, timestamp=timestamp, windows=windows) 568 569 def _flush(self): 570 # Create thread pool executor and process the buffered elements in paralllel 571 executor = ThreadPoolExecutor(max_workers=self.max_workers) 572 futures = [ 573 executor.submit(self.process_buffer_element, ele) for ele in self.buffer 574 ] 575 self.buffer = [] 576 for f in as_completed(futures): 577 yield f.result()