-
Notifications
You must be signed in to change notification settings - Fork 55
/
Copy pathpipeline_common.py
344 lines (294 loc) · 12.9 KB
/
pipeline_common.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
# Copyright 2018 Google Inc. All Rights Reserved.
#
# Licensed 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.
"""Common functions used by vcf_to_bq, bq_to_vcf and vcf_to_bq_preprocessor.
It includes parsing the command line arguments, reading the input, applying the
PTransforms and writing the output.
"""
from typing import List # pylint: disable=unused-import
import argparse
import enum
import os
import uuid
from datetime import datetime
import apache_beam as beam
from apache_beam import pvalue # pylint: disable=unused-import
from apache_beam.io import filesystem
from apache_beam.io import filesystems
from apache_beam.options import pipeline_options
from apache_beam.runners.direct import direct_runner
from gcp_variant_transforms.beam_io import bgzf_io
from gcp_variant_transforms.beam_io import vcf_estimate_io
from gcp_variant_transforms.beam_io import vcf_header_io
from gcp_variant_transforms.beam_io import vcf_parser
from gcp_variant_transforms.beam_io import vcfio
from gcp_variant_transforms.transforms import fusion_break
from gcp_variant_transforms.transforms import merge_headers
# If the # of files matching the input file_pattern exceeds this value, then
# headers will be merged in beam.
_SMALL_DATA_THRESHOLD = 100
_LARGE_DATA_THRESHOLD = 50000
_DATAFLOW_RUNNER_ARG_VALUE = 'DataflowRunner'
SampleNameEncoding = vcf_parser.SampleNameEncoding
class PipelineModes(enum.Enum):
"""An Enum specifying the mode of the pipeline based on the data size."""
SMALL = 0
MEDIUM = 1
LARGE = 2
def parse_args(argv, command_line_options):
# type: (List[str], List[type]) -> (argparse.Namespace, List[str])
"""Parses the arguments.
Args:
argv: A list of string representing the pipeline arguments.
command_line_options: A list of type ``VariantTransformsOptions`` that
specifies the options that will be added to parser.
"""
parser = argparse.ArgumentParser()
parser.register('type', 'bool', lambda v: v.lower() == 'true')
options = [option() for option in command_line_options]
for transform_options in options:
transform_options.add_arguments(parser)
known_args, pipeline_args = parser.parse_known_args(argv)
for transform_options in options:
transform_options.validate(known_args)
_raise_error_on_invalid_flags(
pipeline_args,
known_args.output_table if hasattr(known_args, 'output_table') else None)
if hasattr(known_args, 'input_pattern') or hasattr(known_args, 'input_file'):
known_args.all_patterns = _get_all_patterns(
known_args.input_pattern, known_args.input_file)
# Enable new BQ sink experiment.
pipeline_args += ['--experiment', 'use_beam_bq_sink']
return known_args, pipeline_args
def _get_all_patterns(input_pattern, input_file):
# type: (str, str) -> List[str]
patterns = [input_pattern] if input_pattern else _get_file_names(input_file)
# Validate inputs.
try:
# Gets at most 1 pattern match result of type `filesystems.MatchResult`.
matches = filesystems.FileSystems.match(patterns, [1] * len(patterns))
for match in matches:
if not match.metadata_list:
if input_file:
raise ValueError(
'Input pattern {} from {} did not match any files.'.format(
match.pattern, input_file))
else:
raise ValueError(
'Input pattern {} did not match any files.'.format(match.pattern))
except filesystem.BeamIOError:
if input_file:
raise ValueError(
'Some patterns in {} are invalid or inaccessible.'.format(
input_file))
else:
raise ValueError('Invalid or inaccessible input pattern {}.'.format(
input_pattern))
return patterns
def get_compression_type(input_patterns):
# type: (List[str]) -> filesystem.CompressionTypes
"""Returns the compression type.
Raises:
ValueError: if the input files are not in the same format.
"""
matches = filesystems.FileSystems.match(input_patterns)
extensions = [os.path.splitext(metadata.path)[1] for match in matches
for metadata in match.metadata_list]
if len(set(extensions)) != 1:
raise ValueError('All input files must be in the same format.')
if extensions[0].endswith('.bgz') or extensions[0].endswith('.gz'):
return filesystem.CompressionTypes.GZIP
else:
return filesystem.CompressionTypes.AUTO
def _get_splittable_bgzf(all_patterns):
# type: (List[str]) -> List[str]
"""Returns the splittable bgzf matching `all_patterns`."""
matches = filesystems.FileSystems.match(all_patterns)
splittable_bgzf = []
count = 0
for match in matches:
for metadata in match.metadata_list:
count += 1
if (metadata.path.startswith('gs://') and
bgzf_io.exists_tbi_file(metadata.path)):
splittable_bgzf.append(metadata.path)
if splittable_bgzf and len(splittable_bgzf) < count:
raise ValueError("Some index files are missing for {}.".format(
all_patterns))
return splittable_bgzf
def _get_file_names(input_file):
# type: (str) -> List[str]
"""Reads the input file and extracts list of patterns out of it."""
if not filesystems.FileSystems.exists(input_file):
raise ValueError('Input file {} doesn\'t exist'.format(input_file))
with filesystems.FileSystems.open(input_file) as f:
contents = map(str.strip, f.readlines())
if not contents:
raise ValueError('Input file {} is empty.'.format(input_file))
return contents
def get_pipeline_mode(all_patterns, optimize_for_large_inputs=False):
# type: (List[str], bool) -> int
"""Returns the mode the pipeline should operate in based on input size."""
if optimize_for_large_inputs or len(all_patterns) > 1:
return PipelineModes.LARGE
match_results = filesystems.FileSystems.match(all_patterns)
if not match_results:
raise ValueError(
'No files matched input_pattern: {}'.format(all_patterns[0]))
total_files = len(match_results[0].metadata_list)
if total_files > _LARGE_DATA_THRESHOLD:
return PipelineModes.LARGE
elif total_files > _SMALL_DATA_THRESHOLD:
return PipelineModes.MEDIUM
return PipelineModes.SMALL
def get_estimates(pipeline, pipeline_mode, all_patterns):
# type: (beam.Pipeline, int, List[str]) -> pvalue.PCollection
"""Creates a PCollection by reading the VCF files and deriving estimates."""
if pipeline_mode == PipelineModes.LARGE:
estimates = (pipeline
| beam.Create(all_patterns)
| vcf_estimate_io.GetAllEstimates())
else:
estimates = pipeline | vcf_estimate_io.GetEstimates(all_patterns[0])
return estimates
def read_headers(
pipeline, #type: beam.Pipeline
pipeline_mode, #type: int
all_patterns #type: List[str]
):
# type: (...) -> pvalue.PCollection
"""Creates an initial PCollection by reading the VCF file headers."""
compression_type = get_compression_type(all_patterns)
if pipeline_mode == PipelineModes.LARGE:
headers = (pipeline
| beam.Create(all_patterns)
| vcf_header_io.ReadAllVcfHeaders(
compression_type=compression_type))
else:
headers = pipeline | vcf_header_io.ReadVcfHeaders(
all_patterns[0],
compression_type=compression_type)
return headers
def read_variants(
pipeline, # type: beam.Pipeline
all_patterns, # type: List[str]
pipeline_mode, # type: PipelineModes
allow_malformed_records, # type: bool
representative_header_lines=None, # type: List[str]
pre_infer_headers=False, # type: bool
sample_name_encoding=SampleNameEncoding.WITHOUT_FILE_PATH # type: int
):
# type: (...) -> pvalue.PCollection
"""Returns a PCollection of Variants by reading VCFs."""
compression_type = get_compression_type(all_patterns)
if compression_type == filesystem.CompressionTypes.GZIP:
splittable_bgzf = _get_splittable_bgzf(all_patterns)
if splittable_bgzf:
return (pipeline
| 'ReadVariants'
>> vcfio.ReadFromBGZF(splittable_bgzf,
representative_header_lines,
allow_malformed_records,
pre_infer_headers,
sample_name_encoding))
if pipeline_mode == PipelineModes.LARGE:
variants = (pipeline
| 'InputFilePattern' >> beam.Create(all_patterns)
| 'ReadAllFromVcf' >> vcfio.ReadAllFromVcf(
representative_header_lines=representative_header_lines,
compression_type=compression_type,
allow_malformed_records=allow_malformed_records,
pre_infer_headers=pre_infer_headers,
sample_name_encoding=sample_name_encoding))
else:
variants = pipeline | 'ReadFromVcf' >> vcfio.ReadFromVcf(
all_patterns[0],
representative_header_lines=representative_header_lines,
compression_type=compression_type,
allow_malformed_records=allow_malformed_records,
pre_infer_headers=pre_infer_headers,
sample_name_encoding=sample_name_encoding)
if compression_type == filesystem.CompressionTypes.GZIP:
variants |= 'FusionBreak' >> fusion_break.FusionBreak()
return variants
def add_annotation_headers(pipeline, known_args, pipeline_mode,
merged_header,
annotated_vcf_pattern):
if pipeline_mode == PipelineModes.LARGE:
annotation_headers = (pipeline
| 'ReadAnnotatedVCF'
>> beam.Create([annotated_vcf_pattern])
| 'ReadHeaders' >> vcf_header_io.ReadAllVcfHeaders())
else:
annotation_headers = (
pipeline
| 'ReadHeaders'
>> vcf_header_io.ReadVcfHeaders(annotated_vcf_pattern))
merged_header = (
(merged_header, annotation_headers)
| beam.Flatten()
| 'MergeWithOriginalHeaders' >> merge_headers.MergeHeaders(
known_args.split_alternate_allele_info_fields,
known_args.allow_incompatible_records))
return merged_header
def get_merged_headers(headers,
split_alternate_allele_info_fields=True,
allow_incompatible_records=True):
# type: (pvalue.PCollection, bool, bool) -> pvalue.PCollection
"""Applies the ``MergeHeaders`` PTransform on PCollection of ``VcfHeader``.
Args:
headers: The VCF headers.
split_alternate_allele_info_fields: If true, the INFO fields with `Number=A`
in BigQuery schema is not repeated. This is relevant as it changes the
header compatibility rules.
allow_incompatible_records: If true, always resolve the conflicts when
merging headers.
"""
return (headers | 'MergeHeaders' >> merge_headers.MergeHeaders(
split_alternate_allele_info_fields, allow_incompatible_records))
def write_headers(merged_header, file_path):
# type: (pvalue.PCollection, str) -> None
"""Writes a PCollection of ``VcfHeader`` to location ``file_path``."""
_ = (merged_header | 'WriteHeaders' >>
vcf_header_io.WriteVcfHeaders(file_path))
def _raise_error_on_invalid_flags(pipeline_args, output_table):
# type: (List[str], Any) -> None
"""Raises an error if there are unrecognized flags."""
parser = argparse.ArgumentParser()
for cls in pipeline_options.PipelineOptions.__subclasses__():
if '_add_argparse_args' in cls.__dict__:
cls._add_argparse_args(parser)
known_pipeline_args, unknown = parser.parse_known_args(pipeline_args)
if unknown:
raise ValueError('Unrecognized flag(s): {}'.format(unknown))
if (known_pipeline_args.runner == _DATAFLOW_RUNNER_ARG_VALUE and
not known_pipeline_args.setup_file):
raise ValueError('The --setup_file flag is required for DataflowRunner. '
'Please provide a path to the setup.py file.')
if output_table:
if (not hasattr(known_pipeline_args, 'temp_location') or
not known_pipeline_args.temp_location):
raise ValueError('--temp_location is required for BigQuery imports.')
if not known_pipeline_args.temp_location.startswith('gs://'):
raise ValueError(
'--temp_location must be valid GCS location for BigQuery imports')
def is_pipeline_direct_runner(pipeline):
# type: (beam.Pipeline) -> bool
"""Returns True if the pipeline's runner is DirectRunner."""
return isinstance(pipeline.runner, direct_runner.DirectRunner)
def generate_unique_name(job_name):
# type: (str) -> str
"""Returns a unique name with time suffix and random UUID."""
return '-'.join([job_name,
datetime.now().strftime('%Y%m%d-%H%M%S'),
str(uuid.uuid4())])