You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ch...@apache.org on 2017/11/08 16:48:10 UTC

[1/4] beam git commit: Added vcf file io source and modified _TextSource to optionally handle headers

Repository: beam
Updated Branches:
  refs/heads/master 9c5454287 -> 0af972095


http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/testing/data/vcf/valid-4.2.vcf
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/testing/data/vcf/valid-4.2.vcf b/sdks/python/apache_beam/testing/data/vcf/valid-4.2.vcf
new file mode 100644
index 0000000..c42d71c
--- /dev/null
+++ b/sdks/python/apache_beam/testing/data/vcf/valid-4.2.vcf
@@ -0,0 +1,42 @@
+##fileformat=VCFv4.2
+##fileDate=20090805
+##source=myImputationProgramV3.1
+##phasing=partial
+##INFO=<ID=NS,Number=1,Type=Integer,Description="Number of Samples With Data">
+##INFO=<ID=DP,Number=1,Type=Integer,Description="Total Depth">
+##INFO=<ID=AF,Number=A,Type=Float,Description="Allele Frequency">
+##INFO=<ID=AA,Number=1,Type=String,Description="Ancestral Allele">
+##INFO=<ID=DB,Number=0,Type=Flag,Description="dbSNP membership, build 129">
+##INFO=<ID=H2,Number=0,Type=Flag,Description="HapMap2 membership">
+##INFO=<ID=SVTYPE,Number=1,Type=String,Description="Type of structural variant (with unïcodé)">
+##INFO=<ID=END,Number=1,Type=Integer,Description="End position of variant">
+##FILTER=<ID=q10,Description="Quality below 10">
+##FILTER=<ID=s50,Description="Less than 50% of samples have data (with \\ backslash)">
+##FORMAT=<ID=GT,Number=1,Type=String,Description="Genotype">
+##FORMAT=<ID=GQ,Number=1,Type=Integer,Description="Genotype Quality">
+##FORMAT=<ID=DP,Number=1,Type=Integer,Description="Read Depth">
+##FORMAT=<ID=HQ,Number=2,Type=Integer,Description="Haplotype Quality">
+##FORMAT=<ID=GL,Number=G,Type=Integer,Description="Genotype Likelihood">
+##reference=file:/lustre/scratch105/projects/g1k/ref/main_project/human_g1k_v37.fasta
+##contig=<ID=19,length=59128983,md5=1aacd71f30db8e561810913e0b72636d,species="Homo Sapiens">
+##contig=<ID=20,length=63025520,md5=0dec9660ec1efaaf33281c0d5ea2560f,species="Homo Sapiens">
+##contig=<ID=Y,length=63025520,md5=0dec9660ec1efaaf33281c0d5ea2560f,species="Homo Sapiens">
+##SAMPLE=<ID=Blood,Genomes=Germline,Mixture=1.,Description="Patient germline genome">
+##SAMPLE=<ID=TissueSample,Genomes=Germline;Tumor,Mixture=.3;.7,Description="Patient germline genome;Patient tumor genome">
+##PEDIGREE=<Derived=ID2,Original=ID1>
+##PEDIGREE=<Child=CHILD-GENOME-ID,Mother=MOTHER-GENOME-ID,Father=FATHER-GENOME-ID>
+##pedigreeDB=url
+#CHROM	POS	ID	REF	ALT	QUAL	FILTER	INFO	FORMAT	NA00001	NA00002	NA00003
+19	14370	rs6054257	G	A	29	PASS	NS=3;DP=14;AF=0.5;DB;H2	GT:GQ:DP:HQ	0|0:48:1:51,51	1|0:48:8:51,51	1/1:43:5:.,.
+20	17330	.	T	A	3	q10	NS=3;DP=11;AF=0.017	GT:GQ:DP:HQ	0|0:49:3:58,50	0|1:3:5:65,3	0/0:41:3
+20	1110696	rs6040355	A	G,T	67	PASS	NS=2;DP=10;AF=0.333,0.667;AA=T;DB	GT:GQ:DP:HQ	1|2:21:6:23,27	2|1:2:0:18,2	2/2:35:4
+20	1230237	.	T	.	47	PASS	NS=3;DP=13;AA=T	GT:GQ:DP:HQ	0|0:54:7:56,60	0|0:48:4:51,51	0/0:61:2
+20	1234567	microsat1	GTC	G,GTCTC	50	PASS	NS=3;DP=9;AA=G	GT:GQ:DP	0/1:35:4	0/2:17:2	1/1:40:3
+20	2234567	.	C	[13:123457[ACGC	50	PASS	SVTYPE=BÑD;NS=3;DP=9;AA=G	GT:GQ:DP	0/1:35:4	0/1:17:2	1/1:40:3
+20	2234568	.	C	.TC	50	PASS	SVTYPE=BND;NS=3;DP=9;AA=G	GT:GQ:DP	0/1:35:4	0/1:17:2	1/1:40:3
+20	2234569	.	C	CT.	50	PASS	SVTYPE=BND;NS=3;DP=9;AA=G	GT:GQ:DP	0/1:35:4	0/1:17:2	1/1:40:3
+20	3234569	.	C	<SYMBOLIC>	50	PASS	END=3235677;NS=3;DP=9;AA=G	GT:GQ:DP	0/1:35:4	0/1:17:2	1/1:40:3
+20	4234569	.	N	.[13:123457[	50	PASS	SVTYPE=BND;NS=3;DP=9;AA=G	GT:GQ:DP	0/1:35:4	0/1:17:2	./.:40:3
+20	5234569	.	N	[13:123457[.	50	PASS	SVTYPE=BND;NS=3;DP=9;AA=G	GT:GQ:DP	0/1:35:4	0/1:17:2	1/1:40:3
+Y	17330	.	T	A	3	q10	NS=3;DP=11;AF=0.017	GT:GL	0:0,49	0:0,3	1:41,0
+HLA-A*01:01:01:01	1	.	N	T	50	PASS	END=1;NS=3;DP=9;AA=G	GT:GQ:DP:HQ	0|0:48:1:51,51	1|0:48:8:51,51	1/1:43:5:.,.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/testing/data/vcf/valid-4.2.vcf.gz
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/testing/data/vcf/valid-4.2.vcf.gz b/sdks/python/apache_beam/testing/data/vcf/valid-4.2.vcf.gz
new file mode 100644
index 0000000..4208e3e
Binary files /dev/null and b/sdks/python/apache_beam/testing/data/vcf/valid-4.2.vcf.gz differ

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/testing/test_utils.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/testing/test_utils.py b/sdks/python/apache_beam/testing/test_utils.py
index 41a02cf..c28b692 100644
--- a/sdks/python/apache_beam/testing/test_utils.py
+++ b/sdks/python/apache_beam/testing/test_utils.py
@@ -22,6 +22,9 @@ For internal use only; no backwards-compatibility guarantees.
 
 import hashlib
 import imp
+import os
+import shutil
+import tempfile
 
 from mock import Mock
 from mock import patch
@@ -32,6 +35,43 @@ from apache_beam.utils import retry
 DEFAULT_HASHING_ALG = 'sha1'
 
 
+class TempDir(object):
+  """Context Manager to create and clean-up a temporary directory."""
+
+  def __init__(self):
+    self._tempdir = tempfile.mkdtemp()
+
+  def __enter__(self):
+    return self
+
+  def __exit__(self, *args):
+    if os.path.exists(self._tempdir):
+      shutil.rmtree(self._tempdir)
+
+  def get_path(self):
+    """Returns the path to the temporary directory."""
+    return self._tempdir
+
+  def create_temp_file(self, suffix='', lines=None):
+    """Creates a temporary file in the temporary directory.
+
+    Args:
+      suffix (str): The filename suffix of the temporary file (e.g. '.txt')
+      lines (List[str]): A list of lines that will be written to the temporary
+        file.
+    Returns:
+      The name of the temporary file created.
+    """
+    f = tempfile.NamedTemporaryFile(delete=False,
+                                    dir=self._tempdir,
+                                    suffix=suffix)
+    if lines:
+      for line in lines:
+        f.write(line)
+
+    return f.name
+
+
 def compute_hash(content, hashing_alg=DEFAULT_HASHING_ALG):
   """Compute a hash value from a list of string."""
   content.sort()

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/testing/test_utils_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/testing/test_utils_test.py b/sdks/python/apache_beam/testing/test_utils_test.py
index 8113310..877ee39 100644
--- a/sdks/python/apache_beam/testing/test_utils_test.py
+++ b/sdks/python/apache_beam/testing/test_utils_test.py
@@ -56,6 +56,30 @@ class TestUtilsTest(unittest.TestCase):
     with self.assertRaises(RuntimeError):
       utils.delete_files([])
 
+  def test_temp_dir_removes_files(self):
+    dir_path = ''
+    file_path = ''
+    with utils.TempDir() as tempdir:
+      dir_path = tempdir.get_path()
+      file_path = tempdir.create_temp_file()
+      self.assertTrue(os.path.exists(dir_path))
+      self.assertTrue(os.path.exists(file_path))
+
+    self.assertFalse(os.path.exists(dir_path))
+    self.assertFalse(os.path.exists(file_path))
+
+  def test_temp_file_field_correct(self):
+    with utils.TempDir() as tempdir:
+      filename = tempdir.create_temp_file(
+          suffix='.txt',
+          lines=['line1\n', 'line2\n', 'line3\n'])
+      self.assertTrue(filename.endswith('.txt'))
+
+      with open(filename, 'rb') as f:
+        self.assertEqual(f.readline(), 'line1\n')
+        self.assertEqual(f.readline(), 'line2\n')
+        self.assertEqual(f.readline(), 'line3\n')
+
 
 if __name__ == '__main__':
   logging.getLogger().setLevel(logging.INFO)

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/generate_pydoc.sh
----------------------------------------------------------------------
diff --git a/sdks/python/generate_pydoc.sh b/sdks/python/generate_pydoc.sh
index 662bd09..9ae019c 100755
--- a/sdks/python/generate_pydoc.sh
+++ b/sdks/python/generate_pydoc.sh
@@ -115,6 +115,9 @@ ignore_identifiers = [
   'Set',
   'Tuple',
 
+  # Ignore broken built-in type references
+  'tuple',
+
   # Ignore private classes
   'apache_beam.coders.coders._PickleCoderBase',
   'apache_beam.coders.coders.FastCoder',
@@ -143,6 +146,9 @@ ignore_identifiers = [
 
   # Private classes which are used within the same module
   'WindowedTypeConstraint',  # apache_beam.typehints.typehints
+
+  # stdlib classes without documentation
+  'unittest.case.TestCase'
 ]
 
 # When inferring a base class it will use ':py:class'; if inferring a function

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/setup.py
----------------------------------------------------------------------
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index 5852d15..46f4f8a 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -104,6 +104,7 @@ REQUIRED_PACKAGES = [
     'oauth2client>=2.0.1,<4.0.0',
     'protobuf>=3.2.0,<=3.3.0',
     'pyyaml>=3.12,<4.0.0',
+    'pyvcf>=0.6.8,<0.7.0',
     # Six 1.11.0 incompatible with apitools.
     # TODO(BEAM-2964): Remove the upper bound.
     'six>=1.9,<1.11',
@@ -160,7 +161,7 @@ setuptools.setup(
     author_email=PACKAGE_EMAIL,
     packages=setuptools.find_packages(),
     package_data={'apache_beam': [
-        '*/*.pyx', '*/*/*.pyx', '*/*.pxd', '*/*/*.pxd', 'testing/data/*']},
+        '*/*.pyx', '*/*/*.pyx', '*/*.pxd', '*/*/*.pxd', 'testing/data/*.yaml']},
     ext_modules=cythonize([
         'apache_beam/**/*.pyx',
         'apache_beam/coders/coder_impl.py',


[3/4] beam git commit: Added vcf file io source and modified _TextSource to optionally handle headers

Posted by ch...@apache.org.
Added vcf file io source and modified _TextSource to optionally handle headers


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/f22da33c
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/f22da33c
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/f22da33c

Branch: refs/heads/master
Commit: f22da33cc46b48c046740cbcfda78ea59adbd924
Parents: 9c54542
Author: Miles Saul <ms...@msaul0.wat.corp.google.com>
Authored: Wed Oct 11 15:00:03 2017 -0400
Committer: chamikara@google.com <ch...@google.com>
Committed: Wed Nov 8 08:46:18 2017 -0800

----------------------------------------------------------------------
 pom.xml                                         |     3 +
 sdks/python/apache_beam/io/source_test_utils.py |     3 +-
 sdks/python/apache_beam/io/textio.py            |    67 +-
 sdks/python/apache_beam/io/textio_test.py       |   503 +-
 sdks/python/apache_beam/io/vcfio.py             |   436 +
 sdks/python/apache_beam/io/vcfio_test.py        |   519 +
 .../apache_beam/testing/data/vcf/valid-4.0.vcf  |    23 +
 .../testing/data/vcf/valid-4.0.vcf.bz2          |   Bin 0 -> 781 bytes
 .../testing/data/vcf/valid-4.0.vcf.gz           |   Bin 0 -> 727 bytes
 .../testing/data/vcf/valid-4.1-large.vcf        | 10000 +++++++++++++++++
 .../testing/data/vcf/valid-4.1-large.vcf.gz     |   Bin 0 -> 156715 bytes
 .../apache_beam/testing/data/vcf/valid-4.2.vcf  |    42 +
 .../testing/data/vcf/valid-4.2.vcf.gz           |   Bin 0 -> 1240 bytes
 sdks/python/apache_beam/testing/test_utils.py   |    40 +
 .../apache_beam/testing/test_utils_test.py      |    24 +
 sdks/python/generate_pydoc.sh                   |     6 +
 sdks/python/setup.py                            |     3 +-
 17 files changed, 11419 insertions(+), 250 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0b7b323..7efb23d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1695,6 +1695,9 @@
               <!-- Proto/grpc generated wrappers -->
               <exclude>**/apache_beam/portability/api/*_pb2*.py</exclude>
               <exclude>**/go/pkg/beam/model/**/*.pb.go</exclude>
+
+              <!-- VCF test files -->
+              <exclude>**/apache_beam/testing/data/vcf/*</exclude>
             </excludes>
           </configuration>
         </plugin>

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/io/source_test_utils.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/source_test_utils.py b/sdks/python/apache_beam/io/source_test_utils.py
index 712049b..e4d2f6f 100644
--- a/sdks/python/apache_beam/io/source_test_utils.py
+++ b/sdks/python/apache_beam/io/source_test_utils.py
@@ -52,7 +52,8 @@ from multiprocessing.pool import ThreadPool
 
 from apache_beam.io import iobase
 
-__all__ = ['read_from_source', 'assert_sources_equal_reference_source',
+__all__ = ['read_from_source',
+           'assert_sources_equal_reference_source',
            'assert_reentrant_reads_succeed',
            'assert_split_at_fraction_behavior',
            'assert_split_at_fraction_binary',

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/io/textio.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py
index c25181d..4a4bd3a 100644
--- a/sdks/python/apache_beam/io/textio.py
+++ b/sdks/python/apache_beam/io/textio.py
@@ -78,6 +78,10 @@ class _TextSource(filebasedsource.FileBasedSource):
                          'size of data %d.', value, len(self._data))
       self._position = value
 
+    def reset(self):
+      self.data = ''
+      self.position = 0
+
   def __init__(self,
                file_pattern,
                min_bundle_size,
@@ -86,7 +90,26 @@ class _TextSource(filebasedsource.FileBasedSource):
                coder,
                buffer_size=DEFAULT_READ_BUFFER_SIZE,
                validate=True,
-               skip_header_lines=0):
+               skip_header_lines=0,
+               header_processor_fns=(None, None)):
+    """Initialize a _TextSource
+
+    Args:
+      header_processor_fns (tuple): a tuple of a `header_matcher` function
+        and a `header_processor` function. The `header_matcher` should
+        return `True` for all lines at the start of the file that are part
+        of the file header and `False` otherwise. These header lines will
+        not be yielded when reading records and instead passed into
+        `header_processor` to be handled. If `skip_header_lines` and a
+        `header_matcher` are both provided, the value of `skip_header_lines`
+        lines will be skipped and the header will be processed from
+        there.
+    Raises:
+      ValueError: if skip_lines is negative.
+
+    Please refer to documentation in class `ReadFromText` for the rest
+    of the arguments.
+    """
     super(_TextSource, self).__init__(file_pattern, min_bundle_size,
                                       compression_type=compression_type,
                                       validate=validate)
@@ -103,6 +126,7 @@ class _TextSource(filebasedsource.FileBasedSource):
           'Skipping %d header lines. Skipping large number of header '
           'lines might significantly slow down processing.')
     self._skip_header_lines = skip_header_lines
+    self._header_matcher, self._header_processor = header_processor_fns
 
   def display_data(self):
     parent_dd = super(_TextSource, self).display_data()
@@ -130,18 +154,17 @@ class _TextSource(filebasedsource.FileBasedSource):
     range_tracker.set_split_points_unclaimed_callback(split_points_unclaimed)
 
     with self.open_file(file_name) as file_to_read:
-      position_after_skipping_header_lines = self._skip_lines(
-          file_to_read, read_buffer,
-          self._skip_header_lines) if self._skip_header_lines else 0
-      start_offset = max(start_offset, position_after_skipping_header_lines)
-      if start_offset > position_after_skipping_header_lines:
+      position_after_processing_header_lines = (
+          self._process_header(file_to_read, read_buffer))
+      start_offset = max(start_offset, position_after_processing_header_lines)
+      if start_offset > position_after_processing_header_lines:
         # Seeking to one position before the start index and ignoring the
         # current line. If start_position is at beginning if the line, that line
         # belongs to the current bundle, hence ignoring that is incorrect.
         # Seeking to one byte before prevents that.
 
         file_to_read.seek(start_offset - 1)
-        read_buffer = _TextSource.ReadBuffer('', 0)
+        read_buffer.reset()
         sep_bounds = self._find_separator_bounds(file_to_read, read_buffer)
         if not sep_bounds:
           # Could not find a separator after (start_offset - 1). This means that
@@ -152,7 +175,7 @@ class _TextSource(filebasedsource.FileBasedSource):
         read_buffer.data = read_buffer.data[sep_end:]
         next_record_start_position = start_offset - 1 + sep_end
       else:
-        next_record_start_position = position_after_skipping_header_lines
+        next_record_start_position = position_after_processing_header_lines
 
       while range_tracker.try_claim(next_record_start_position):
         record, num_bytes_to_next_record = self._read_record(file_to_read,
@@ -175,6 +198,34 @@ class _TextSource(filebasedsource.FileBasedSource):
         if num_bytes_to_next_record < 0:
           break
 
+  def _process_header(self, file_to_read, read_buffer):
+    # Returns a tuple containing the position in file after processing header
+    # records and a list of decoded header lines that match
+    # 'header_matcher'.
+    header_lines = []
+    position = self._skip_lines(
+        file_to_read, read_buffer,
+        self._skip_header_lines) if self._skip_header_lines else 0
+    if self._header_matcher:
+      while True:
+        record, num_bytes_to_next_record = self._read_record(file_to_read,
+                                                             read_buffer)
+        decoded_line = self._coder.decode(record)
+        if not self._header_matcher(decoded_line):
+          # We've read past the header section at this point, so go back a line.
+          file_to_read.seek(position)
+          read_buffer.reset()
+          break
+        header_lines.append(decoded_line)
+        if num_bytes_to_next_record < 0:
+          break
+        position += num_bytes_to_next_record
+
+      if self._header_processor:
+        self._header_processor(header_lines)
+
+    return position
+
   def _find_separator_bounds(self, file_to_read, read_buffer):
     # Determines the start and end positions within 'read_buffer.data' of the
     # next separator starting from position 'read_buffer.position'.

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/io/textio_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py
index 475848f..324f52a 100644
--- a/sdks/python/apache_beam/io/textio_test.py
+++ b/sdks/python/apache_beam/io/textio_test.py
@@ -41,44 +41,13 @@ from apache_beam.io.textio import _TextSource as TextSource
 from apache_beam.io.textio import ReadFromText
 from apache_beam.io.textio import WriteToText
 from apache_beam.testing.test_pipeline import TestPipeline
+from apache_beam.testing.test_utils import TempDir
 from apache_beam.testing.util import assert_that
 from apache_beam.testing.util import equal_to
 from apache_beam.transforms.core import Create
 
 
-# TODO: Refactor code so all io tests are using same library
-# TestCaseWithTempDirCleanup class.
-class _TestCaseWithTempDirCleanUp(unittest.TestCase):
-  """Base class for TestCases that deals with TempDir clean-up.
-
-  Inherited test cases will call self._new_tempdir() to start a temporary dir
-  which will be deleted at the end of the tests (when tearDown() is called).
-  """
-
-  def setUp(self):
-    self._tempdirs = []
-
-  def tearDown(self):
-    for path in self._tempdirs:
-      if os.path.exists(path):
-        shutil.rmtree(path)
-    self._tempdirs = []
-
-  def _new_tempdir(self):
-    result = tempfile.mkdtemp()
-    self._tempdirs.append(result)
-    return result
-
-  def _create_temp_file(self, name='', suffix=''):
-    if not name:
-      name = tempfile.template
-    file_name = tempfile.NamedTemporaryFile(
-        delete=False, prefix=name,
-        dir=self._new_tempdir(), suffix=suffix).name
-    return file_name
-
-
-class TextSourceTest(_TestCaseWithTempDirCleanUp):
+class TextSourceTest(unittest.TestCase):
 
   # Number of records that will be written by most tests.
   DEFAULT_NUM_RECORDS = 100
@@ -92,7 +61,7 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
     source = TextSource(file_or_pattern, 0, compression,
                         True, coders.StrUtf8Coder(), buffer_size)
     range_tracker = source.get_range_tracker(None, None)
-    read_data = [record for record in source.read(range_tracker)]
+    read_data = list(source.read(range_tracker))
     self.assertItemsEqual(expected_data, read_data)
 
   def test_read_single_file(self):
@@ -214,7 +183,7 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
                         False, coders.StrUtf8Coder())
 
     range_tracker = source.get_range_tracker(None, None)
-    read_data = [record for record in source.read(range_tracker)]
+    read_data = list(source.read(range_tracker))
     self.assertItemsEqual([line + '\n' for line in written_data], read_data)
 
   def test_read_single_file_without_striping_eol_crlf(self):
@@ -225,7 +194,7 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
                         False, coders.StrUtf8Coder())
 
     range_tracker = source.get_range_tracker(None, None)
-    read_data = [record for record in source.read(range_tracker)]
+    read_data = list(source.read(range_tracker))
     self.assertItemsEqual([line + '\r\n' for line in written_data], read_data)
 
   def test_read_file_pattern_with_empty_files(self):
@@ -246,7 +215,7 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
     assert len(expected_data) == 10
     source = TextSource(file_name, 0, CompressionTypes.UNCOMPRESSED, True,
                         coders.StrUtf8Coder())
-    splits = [split for split in source.split(desired_bundle_size=33)]
+    splits = list(source.split(desired_bundle_size=33))
 
     reference_source_info = (source, None, None)
     sources_info = ([
@@ -255,12 +224,37 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
     source_test_utils.assert_sources_equal_reference_source(
         reference_source_info, sources_info)
 
+  def test_header_processing(self):
+    file_name, expected_data = write_data(10)
+    assert len(expected_data) == 10
+
+    def header_matcher(line):
+      return line in expected_data[:5]
+
+    header_lines = []
+
+    def store_header(lines):
+      for line in lines:
+        header_lines.append(line)
+
+    source = TextSource(file_name, 0, CompressionTypes.UNCOMPRESSED, True,
+                        coders.StrUtf8Coder(),
+                        header_processor_fns=(header_matcher, store_header))
+    splits = list(source.split(desired_bundle_size=100000))
+    assert len(splits) == 1
+    range_tracker = splits[0].source.get_range_tracker(
+        splits[0].start_position, splits[0].stop_position)
+    read_data = list(source.read_records(file_name, range_tracker))
+
+    self.assertItemsEqual(expected_data[:5], header_lines)
+    self.assertItemsEqual(expected_data[5:], read_data)
+
   def test_progress(self):
     file_name, expected_data = write_data(10)
     assert len(expected_data) == 10
     source = TextSource(file_name, 0, CompressionTypes.UNCOMPRESSED, True,
                         coders.StrUtf8Coder())
-    splits = [split for split in source.split(desired_bundle_size=100000)]
+    splits = list(source.split(desired_bundle_size=100000))
     assert len(splits) == 1
     fraction_consumed_report = []
     split_points_report = []
@@ -295,7 +289,7 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
     assert len(expected_data) == 10
     source = TextSource(file_name, 0, CompressionTypes.UNCOMPRESSED, True,
                         coders.StrUtf8Coder())
-    splits = [split for split in source.split(desired_bundle_size=100000)]
+    splits = list(source.split(desired_bundle_size=100000))
     assert len(splits) == 1
     source_test_utils.assert_reentrant_reads_succeed(
         (splits[0].source, splits[0].start_position, splits[0].stop_position))
@@ -305,7 +299,7 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
     assert len(expected_data) == 5
     source = TextSource(file_name, 0, CompressionTypes.UNCOMPRESSED, True,
                         coders.StrUtf8Coder())
-    splits = [split for split in source.split(desired_bundle_size=100000)]
+    splits = list(source.split(desired_bundle_size=100000))
     assert len(splits) == 1
     source_test_utils.assert_split_at_fraction_exhaustive(
         splits[0].source, splits[0].start_position, splits[0].stop_position)
@@ -315,7 +309,7 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
     assert len(expected_data) == 15
     source = TextSource(file_name, 0, CompressionTypes.UNCOMPRESSED, True,
                         coders.StrUtf8Coder())
-    splits = [split for split in source.split(desired_bundle_size=100000)]
+    splits = list(source.split(desired_bundle_size=100000))
     assert len(splits) == 1
     source_test_utils.assert_split_at_fraction_exhaustive(
         splits[0].source, splits[0].start_position, splits[0].stop_position,
@@ -326,7 +320,7 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
     assert len(expected_data) == 5
     source = TextSource(file_name, 0, CompressionTypes.UNCOMPRESSED, True,
                         coders.StrUtf8Coder())
-    splits = [split for split in source.split(desired_bundle_size=100000)]
+    splits = list(source.split(desired_bundle_size=100000))
     assert len(splits) == 1
     source_test_utils.assert_split_at_fraction_exhaustive(
         splits[0].source, splits[0].start_position, splits[0].stop_position,
@@ -438,226 +432,241 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
 
   def test_read_auto_bzip2(self):
     _, lines = write_data(15)
-    file_name = self._create_temp_file(suffix='.bz2')
-    with bz2.BZ2File(file_name, 'wb') as f:
-      f.write('\n'.join(lines))
-
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromText(file_name)
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TempDir() as tempdir:
+      file_name = tempdir.create_temp_file(suffix='.bz2')
+      with bz2.BZ2File(file_name, 'wb') as f:
+        f.write('\n'.join(lines))
+
+      pipeline = TestPipeline()
+      pcoll = pipeline | 'Read' >> ReadFromText(file_name)
+      assert_that(pcoll, equal_to(lines))
+      pipeline.run()
 
   def test_read_auto_gzip(self):
     _, lines = write_data(15)
-    file_name = self._create_temp_file(suffix='.gz')
+    with TempDir() as tempdir:
+      file_name = tempdir.create_temp_file(suffix='.gz')
 
-    with gzip.GzipFile(file_name, 'wb') as f:
-      f.write('\n'.join(lines))
+      with gzip.GzipFile(file_name, 'wb') as f:
+        f.write('\n'.join(lines))
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromText(file_name)
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+      pipeline = TestPipeline()
+      pcoll = pipeline | 'Read' >> ReadFromText(file_name)
+      assert_that(pcoll, equal_to(lines))
+      pipeline.run()
 
   def test_read_bzip2(self):
     _, lines = write_data(15)
-    file_name = self._create_temp_file()
-    with bz2.BZ2File(file_name, 'wb') as f:
-      f.write('\n'.join(lines))
-
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromText(
-        file_name,
-        compression_type=CompressionTypes.BZIP2)
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TempDir() as tempdir:
+      file_name = tempdir.create_temp_file()
+      with bz2.BZ2File(file_name, 'wb') as f:
+        f.write('\n'.join(lines))
+
+      pipeline = TestPipeline()
+      pcoll = pipeline | 'Read' >> ReadFromText(
+          file_name,
+          compression_type=CompressionTypes.BZIP2)
+      assert_that(pcoll, equal_to(lines))
+      pipeline.run()
 
   def test_read_corrupted_bzip2_fails(self):
     _, lines = write_data(15)
-    file_name = self._create_temp_file()
-    with bz2.BZ2File(file_name, 'wb') as f:
-      f.write('\n'.join(lines))
-
-    with open(file_name, 'wb') as f:
-      f.write('corrupt')
-
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromText(
-        file_name,
-        compression_type=CompressionTypes.BZIP2)
-    assert_that(pcoll, equal_to(lines))
-    with self.assertRaises(Exception):
-      pipeline.run()
+    with TempDir() as tempdir:
+      file_name = tempdir.create_temp_file()
+      with bz2.BZ2File(file_name, 'wb') as f:
+        f.write('\n'.join(lines))
+
+      with open(file_name, 'wb') as f:
+        f.write('corrupt')
+
+      pipeline = TestPipeline()
+      pcoll = pipeline | 'Read' >> ReadFromText(
+          file_name,
+          compression_type=CompressionTypes.BZIP2)
+      assert_that(pcoll, equal_to(lines))
+      with self.assertRaises(Exception):
+        pipeline.run()
 
   def test_read_bzip2_concat(self):
-    bzip2_file_name1 = self._create_temp_file()
-    lines = ['a', 'b', 'c']
-    with bz2.BZ2File(bzip2_file_name1, 'wb') as dst:
-      data = '\n'.join(lines) + '\n'
-      dst.write(data)
-
-    bzip2_file_name2 = self._create_temp_file()
-    lines = ['p', 'q', 'r']
-    with bz2.BZ2File(bzip2_file_name2, 'wb') as dst:
-      data = '\n'.join(lines) + '\n'
-      dst.write(data)
-
-    bzip2_file_name3 = self._create_temp_file()
-    lines = ['x', 'y', 'z']
-    with bz2.BZ2File(bzip2_file_name3, 'wb') as dst:
-      data = '\n'.join(lines) + '\n'
-      dst.write(data)
-
-    final_bzip2_file = self._create_temp_file()
-    with open(bzip2_file_name1, 'rb') as src, open(
-        final_bzip2_file, 'wb') as dst:
-      dst.writelines(src.readlines())
-
-    with open(bzip2_file_name2, 'rb') as src, open(
-        final_bzip2_file, 'ab') as dst:
-      dst.writelines(src.readlines())
-
-    with open(bzip2_file_name3, 'rb') as src, open(
-        final_bzip2_file, 'ab') as dst:
-      dst.writelines(src.readlines())
-
-    pipeline = TestPipeline()
-    lines = pipeline | 'ReadFromText' >> beam.io.ReadFromText(
-        final_bzip2_file,
-        compression_type=beam.io.filesystem.CompressionTypes.BZIP2)
-
-    expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z']
-    assert_that(lines, equal_to(expected))
-    pipeline.run()
+    with TempDir() as tempdir:
+      bzip2_file_name1 = tempdir.create_temp_file()
+      lines = ['a', 'b', 'c']
+      with bz2.BZ2File(bzip2_file_name1, 'wb') as dst:
+        data = '\n'.join(lines) + '\n'
+        dst.write(data)
+
+      bzip2_file_name2 = tempdir.create_temp_file()
+      lines = ['p', 'q', 'r']
+      with bz2.BZ2File(bzip2_file_name2, 'wb') as dst:
+        data = '\n'.join(lines) + '\n'
+        dst.write(data)
+
+      bzip2_file_name3 = tempdir.create_temp_file()
+      lines = ['x', 'y', 'z']
+      with bz2.BZ2File(bzip2_file_name3, 'wb') as dst:
+        data = '\n'.join(lines) + '\n'
+        dst.write(data)
+
+      final_bzip2_file = tempdir.create_temp_file()
+      with open(bzip2_file_name1, 'rb') as src, open(
+          final_bzip2_file, 'wb') as dst:
+        dst.writelines(src.readlines())
+
+      with open(bzip2_file_name2, 'rb') as src, open(
+          final_bzip2_file, 'ab') as dst:
+        dst.writelines(src.readlines())
+
+      with open(bzip2_file_name3, 'rb') as src, open(
+          final_bzip2_file, 'ab') as dst:
+        dst.writelines(src.readlines())
+
+      pipeline = TestPipeline()
+      lines = pipeline | 'ReadFromText' >> beam.io.ReadFromText(
+          final_bzip2_file,
+          compression_type=beam.io.filesystem.CompressionTypes.BZIP2)
+
+      expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z']
+      assert_that(lines, equal_to(expected))
+      pipeline.run()
 
   def test_read_gzip(self):
     _, lines = write_data(15)
-    file_name = self._create_temp_file()
-    with gzip.GzipFile(file_name, 'wb') as f:
-      f.write('\n'.join(lines))
-
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromText(
-        file_name,
-        0, CompressionTypes.GZIP,
-        True, coders.StrUtf8Coder())
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TempDir() as tempdir:
+      file_name = tempdir.create_temp_file()
+      with gzip.GzipFile(file_name, 'wb') as f:
+        f.write('\n'.join(lines))
+
+      pipeline = TestPipeline()
+      pcoll = pipeline | 'Read' >> ReadFromText(
+          file_name,
+          0, CompressionTypes.GZIP,
+          True, coders.StrUtf8Coder())
+      assert_that(pcoll, equal_to(lines))
+      pipeline.run()
 
   def test_read_corrupted_gzip_fails(self):
     _, lines = write_data(15)
-    file_name = self._create_temp_file()
-    with gzip.GzipFile(file_name, 'wb') as f:
-      f.write('\n'.join(lines))
+    with TempDir() as tempdir:
+      file_name = tempdir.create_temp_file()
+      with gzip.GzipFile(file_name, 'wb') as f:
+        f.write('\n'.join(lines))
 
-    with open(file_name, 'wb') as f:
-      f.write('corrupt')
+      with open(file_name, 'wb') as f:
+        f.write('corrupt')
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromText(
-        file_name,
-        0, CompressionTypes.GZIP,
-        True, coders.StrUtf8Coder())
-    assert_that(pcoll, equal_to(lines))
+      pipeline = TestPipeline()
+      pcoll = pipeline | 'Read' >> ReadFromText(
+          file_name,
+          0, CompressionTypes.GZIP,
+          True, coders.StrUtf8Coder())
+      assert_that(pcoll, equal_to(lines))
 
-    with self.assertRaises(Exception):
-      pipeline.run()
+      with self.assertRaises(Exception):
+        pipeline.run()
 
   def test_read_gzip_concat(self):
-    gzip_file_name1 = self._create_temp_file()
-    lines = ['a', 'b', 'c']
-    with gzip.open(gzip_file_name1, 'wb') as dst:
-      data = '\n'.join(lines) + '\n'
-      dst.write(data)
-
-    gzip_file_name2 = self._create_temp_file()
-    lines = ['p', 'q', 'r']
-    with gzip.open(gzip_file_name2, 'wb') as dst:
-      data = '\n'.join(lines) + '\n'
-      dst.write(data)
-
-    gzip_file_name3 = self._create_temp_file()
-    lines = ['x', 'y', 'z']
-    with gzip.open(gzip_file_name3, 'wb') as dst:
-      data = '\n'.join(lines) + '\n'
-      dst.write(data)
-
-    final_gzip_file = self._create_temp_file()
-    with open(gzip_file_name1, 'rb') as src, open(final_gzip_file, 'wb') as dst:
-      dst.writelines(src.readlines())
-
-    with open(gzip_file_name2, 'rb') as src, open(final_gzip_file, 'ab') as dst:
-      dst.writelines(src.readlines())
-
-    with open(gzip_file_name3, 'rb') as src, open(final_gzip_file, 'ab') as dst:
-      dst.writelines(src.readlines())
-
-    pipeline = TestPipeline()
-    lines = pipeline | 'ReadFromText' >> beam.io.ReadFromText(
-        final_gzip_file,
-        compression_type=beam.io.filesystem.CompressionTypes.GZIP)
-
-    expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z']
-    assert_that(lines, equal_to(expected))
+    with TempDir() as tempdir:
+      gzip_file_name1 = tempdir.create_temp_file()
+      lines = ['a', 'b', 'c']
+      with gzip.open(gzip_file_name1, 'wb') as dst:
+        data = '\n'.join(lines) + '\n'
+        dst.write(data)
+
+      gzip_file_name2 = tempdir.create_temp_file()
+      lines = ['p', 'q', 'r']
+      with gzip.open(gzip_file_name2, 'wb') as dst:
+        data = '\n'.join(lines) + '\n'
+        dst.write(data)
+
+      gzip_file_name3 = tempdir.create_temp_file()
+      lines = ['x', 'y', 'z']
+      with gzip.open(gzip_file_name3, 'wb') as dst:
+        data = '\n'.join(lines) + '\n'
+        dst.write(data)
+
+      final_gzip_file = tempdir.create_temp_file()
+      with open(gzip_file_name1, 'rb') as src, \
+           open(final_gzip_file, 'wb') as dst:
+        dst.writelines(src.readlines())
+
+      with open(gzip_file_name2, 'rb') as src, \
+           open(final_gzip_file, 'ab') as dst:
+        dst.writelines(src.readlines())
+
+      with open(gzip_file_name3, 'rb') as src, \
+           open(final_gzip_file, 'ab') as dst:
+        dst.writelines(src.readlines())
+
+      pipeline = TestPipeline()
+      lines = pipeline | 'ReadFromText' >> beam.io.ReadFromText(
+          final_gzip_file,
+          compression_type=beam.io.filesystem.CompressionTypes.GZIP)
+
+      expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z']
+      assert_that(lines, equal_to(expected))
 
   def test_read_all_gzip(self):
     _, lines = write_data(100)
-    file_name = self._create_temp_file()
-    with gzip.GzipFile(file_name, 'wb') as f:
-      f.write('\n'.join(lines))
-    pipeline = TestPipeline()
-    pcoll = (pipeline
-             | Create([file_name])
-             | 'ReadAll' >> ReadAllFromText(
-                 compression_type=CompressionTypes.GZIP))
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TempDir() as tempdir:
+      file_name = tempdir.create_temp_file()
+      with gzip.GzipFile(file_name, 'wb') as f:
+        f.write('\n'.join(lines))
+      pipeline = TestPipeline()
+      pcoll = (pipeline
+               | Create([file_name])
+               | 'ReadAll' >> ReadAllFromText(
+                   compression_type=CompressionTypes.GZIP))
+      assert_that(pcoll, equal_to(lines))
+      pipeline.run()
 
   def test_read_gzip_large(self):
     _, lines = write_data(10000)
-    file_name = self._create_temp_file()
-
-    with gzip.GzipFile(file_name, 'wb') as f:
-      f.write('\n'.join(lines))
-
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromText(
-        file_name,
-        0, CompressionTypes.GZIP,
-        True, coders.StrUtf8Coder())
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TempDir() as tempdir:
+      file_name = tempdir.create_temp_file()
+
+      with gzip.GzipFile(file_name, 'wb') as f:
+        f.write('\n'.join(lines))
+
+      pipeline = TestPipeline()
+      pcoll = pipeline | 'Read' >> ReadFromText(
+          file_name,
+          0, CompressionTypes.GZIP,
+          True, coders.StrUtf8Coder())
+      assert_that(pcoll, equal_to(lines))
+      pipeline.run()
 
   def test_read_gzip_large_after_splitting(self):
     _, lines = write_data(10000)
-    file_name = self._create_temp_file()
-    with gzip.GzipFile(file_name, 'wb') as f:
-      f.write('\n'.join(lines))
-
-    source = TextSource(file_name, 0, CompressionTypes.GZIP, True,
-                        coders.StrUtf8Coder())
-    splits = [split for split in source.split(desired_bundle_size=1000)]
-
-    if len(splits) > 1:
-      raise ValueError('FileBasedSource generated more than one initial split '
-                       'for a compressed file.')
-
-    reference_source_info = (source, None, None)
-    sources_info = ([
-        (split.source, split.start_position, split.stop_position) for
-        split in splits])
-    source_test_utils.assert_sources_equal_reference_source(
-        reference_source_info, sources_info)
+    with TempDir() as tempdir:
+      file_name = tempdir.create_temp_file()
+      with gzip.GzipFile(file_name, 'wb') as f:
+        f.write('\n'.join(lines))
+
+      source = TextSource(file_name, 0, CompressionTypes.GZIP, True,
+                          coders.StrUtf8Coder())
+      splits = list(source.split(desired_bundle_size=1000))
+
+      if len(splits) > 1:
+        raise ValueError('FileBasedSource generated more than one initial '
+                         'split for a compressed file.')
+
+      reference_source_info = (source, None, None)
+      sources_info = ([
+          (split.source, split.start_position, split.stop_position) for
+          split in splits])
+      source_test_utils.assert_sources_equal_reference_source(
+          reference_source_info, sources_info)
 
   def test_read_gzip_empty_file(self):
-    file_name = self._create_temp_file()
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromText(
-        file_name,
-        0, CompressionTypes.GZIP,
-        True, coders.StrUtf8Coder())
-    assert_that(pcoll, equal_to([]))
-    pipeline.run()
+    with TempDir() as tempdir:
+      file_name = tempdir.create_temp_file()
+      pipeline = TestPipeline()
+      pcoll = pipeline | 'Read' >> ReadFromText(
+          file_name,
+          0, CompressionTypes.GZIP,
+          True, coders.StrUtf8Coder())
+      assert_that(pcoll, equal_to([]))
+      pipeline.run()
 
   def _remove_lines(self, lines, sublist_lengths, num_to_remove):
     """Utility function to remove num_to_remove lines from each sublist.
@@ -690,7 +699,7 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
         skip_header_lines=skip_header_lines)
 
     range_tracker = source.get_range_tracker(None, None)
-    return [record for record in source.read(range_tracker)]
+    return list(source.read(range_tracker))
 
   def test_read_skip_header_single(self):
     file_name, expected_data = write_data(TextSourceTest.DEFAULT_NUM_RECORDS)
@@ -735,23 +744,24 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
 
   def test_read_gzip_with_skip_lines(self):
     _, lines = write_data(15)
-    file_name = self._create_temp_file()
-    with gzip.GzipFile(file_name, 'wb') as f:
-      f.write('\n'.join(lines))
-
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromText(
-        file_name, 0, CompressionTypes.GZIP,
-        True, coders.StrUtf8Coder(), skip_header_lines=2)
-    assert_that(pcoll, equal_to(lines[2:]))
-    pipeline.run()
+    with TempDir() as tempdir:
+      file_name = tempdir.create_temp_file()
+      with gzip.GzipFile(file_name, 'wb') as f:
+        f.write('\n'.join(lines))
+
+      pipeline = TestPipeline()
+      pcoll = pipeline | 'Read' >> ReadFromText(
+          file_name, 0, CompressionTypes.GZIP,
+          True, coders.StrUtf8Coder(), skip_header_lines=2)
+      assert_that(pcoll, equal_to(lines[2:]))
+      pipeline.run()
 
   def test_read_after_splitting_skip_header(self):
     file_name, expected_data = write_data(100)
     assert len(expected_data) == 100
     source = TextSource(file_name, 0, CompressionTypes.UNCOMPRESSED, True,
                         coders.StrUtf8Coder(), skip_header_lines=2)
-    splits = [split for split in source.split(desired_bundle_size=33)]
+    splits = list(source.split(desired_bundle_size=33))
 
     reference_source_info = (source, None, None)
     sources_info = ([
@@ -767,13 +777,26 @@ class TextSourceTest(_TestCaseWithTempDirCleanUp):
     self.assertEqual(reference_lines, split_lines)
 
 
-class TextSinkTest(_TestCaseWithTempDirCleanUp):
+class TextSinkTest(unittest.TestCase):
 
   def setUp(self):
     super(TextSinkTest, self).setUp()
     self.lines = ['Line %d' % d for d in range(100)]
+    self.tempdir = tempfile.mkdtemp()
     self.path = self._create_temp_file()
 
+  def tearDown(self):
+    if os.path.exists(self.tempdir):
+      shutil.rmtree(self.tempdir)
+
+  def _create_temp_file(self, name='', suffix=''):
+    if not name:
+      name = tempfile.template
+    file_name = tempfile.NamedTemporaryFile(
+        delete=False, prefix=name,
+        dir=self.tempdir, suffix=suffix).name
+    return file_name
+
   def _write_lines(self, sink, lines):
     f = sink.open(self.path)
     for line in lines:

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/io/vcfio.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/vcfio.py b/sdks/python/apache_beam/io/vcfio.py
new file mode 100644
index 0000000..b877a32
--- /dev/null
+++ b/sdks/python/apache_beam/io/vcfio.py
@@ -0,0 +1,436 @@
+#
+# 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.
+#
+
+"""A source for reading from VCF files (version 4.x).
+
+The 4.2 spec is available at https://samtools.github.io/hts-specs/VCFv4.2.pdf.
+"""
+
+from __future__ import absolute_import
+
+from collections import namedtuple
+
+import vcf
+
+from apache_beam.coders import coders
+from apache_beam.io import filebasedsource
+from apache_beam.io.filesystem import CompressionTypes
+from apache_beam.io.iobase import Read
+from apache_beam.io.textio import _TextSource as TextSource
+from apache_beam.transforms import PTransform
+
+__all__ = ['ReadFromVcf', 'Variant', 'VariantCall', 'VariantInfo']
+
+
+# Stores data about variant INFO fields. The type of 'data' is specified in the
+# VCF headers. 'field_count' is a string that specifies the number of fields
+# that the data type contains. Its value can either be a number representing a
+# constant number of fields, `None` indicating that the value is not set
+# (equivalent to '.' in the VCF file) or one of:
+#   - 'A': one value per alternate allele.
+#   - 'G': one value for each possible genotype.
+#   - 'R': one value for each possible allele (including the reference).
+VariantInfo = namedtuple('VariantInfo', ['data', 'field_count'])
+MISSING_FIELD_VALUE = '.'  # Indicates field is missing in VCF record.
+PASS_FILTER = 'PASS'  # Indicates that all filters have been passed.
+END_INFO_KEY = 'END'  # The info key that explicitly specifies end of a record.
+GENOTYPE_FORMAT_KEY = 'GT'  # The genotype format key in a call.
+PHASESET_FORMAT_KEY = 'PS'  # The phaseset format key.
+DEFAULT_PHASESET_VALUE = '*'  # Default phaseset value if call is phased, but
+                              # no 'PS' is present.
+MISSING_GENOTYPE_VALUE = -1  # Genotype to use when '.' is used in GT field.
+
+
+class Variant(object):
+  """A class to store info about a genomic variant.
+
+  Each object corresponds to a single record in a VCF file.
+  """
+
+  def __init__(self,
+               reference_name=None,
+               start=None,
+               end=None,
+               reference_bases=None,
+               alternate_bases=None,
+               names=None,
+               quality=None,
+               filters=None,
+               info=None,
+               calls=None):
+    """Initialize the :class:`Variant` object.
+
+    Args:
+      reference_name (str): The reference on which this variant occurs
+        (such as `chr20` or `X`). .
+      start (int): The position at which this variant occurs (0-based).
+        Corresponds to the first base of the string of reference bases.
+      end (int): The end position (0-based) of this variant. Corresponds to the
+        first base after the last base in the reference allele.
+      reference_bases (str): The reference bases for this variant.
+      alternate_bases (List[str]): The bases that appear instead of the
+        reference bases.
+      names (List[str]): Names for the variant, for example a RefSNP ID.
+      quality (float): Phred-scaled quality score (-10log10 prob(call is wrong))
+        Higher values imply better quality.
+      filters (List[str]): A list of filters (normally quality filters) this
+        variant has failed. `PASS` indicates this variant has passed all
+        filters.
+      info (dict): A map of additional variant information. The key is specified
+        in the VCF record and the value is of type ``VariantInfo``.
+      calls (list of :class:`VariantCall`): The variant calls for this variant.
+        Each one represents the determination of genotype with respect to this
+        variant.
+    """
+    self.reference_name = reference_name
+    self.start = start
+    self.end = end
+    self.reference_bases = reference_bases
+    self.alternate_bases = alternate_bases or []
+    self.names = names or []
+    self.quality = quality
+    self.filters = filters or []
+    self.info = info or {}
+    self.calls = calls or []
+
+  def __eq__(self, other):
+    return (isinstance(other, Variant) and
+            vars(self) == vars(other))
+
+  def __repr__(self):
+    return ', '.join(
+        [str(s) for s in [self.reference_name,
+                          self.start,
+                          self.end,
+                          self.reference_bases,
+                          self.alternate_bases,
+                          self.names,
+                          self.quality,
+                          self.filters,
+                          self.info,
+                          self.calls]])
+
+  def __lt__(self, other):
+    if not isinstance(other, Variant):
+      return NotImplemented
+
+    # Elements should first be sorted by reference_name, start, end.
+    # Ordering of other members is not important, but must be
+    # deterministic.
+    if self.reference_name != other.reference_name:
+      return self.reference_name < other.reference_name
+    elif self.start != other.start:
+      return self.start < other.start
+    elif self.end != other.end:
+      return self.end < other.end
+
+    self_vars = vars(self)
+    other_vars = vars(other)
+    for key in sorted(self_vars):
+      if self_vars[key] != other_vars[key]:
+        return self_vars[key] < other_vars[key]
+
+    return False
+
+  def __le__(self, other):
+    if not isinstance(other, Variant):
+      return NotImplemented
+
+    return self < other or self == other
+
+  def __ne__(self, other):
+    return not self == other
+
+  def __gt__(self, other):
+    if not isinstance(other, Variant):
+      return NotImplemented
+
+    return other < self
+
+  def __ge__(self, other):
+    if not isinstance(other, Variant):
+      return NotImplemented
+
+    return other <= self
+
+
+class VariantCall(object):
+  """A class to store info about a variant call.
+
+  A call represents the determination of genotype with respect to a particular
+  variant. It may include associated information such as quality and phasing.
+  """
+
+  def __init__(self, name=None, genotype=None, phaseset=None, info=None):
+    """Initialize the :class:`VariantCall` object.
+
+    Args:
+      name (str): The name of the call.
+      genotype (List[int]): The genotype of this variant call as specified by
+        the VCF schema. The values are either `0` representing the reference,
+        or a 1-based index into alternate bases. Ordering is only important if
+        `phaseset` is present. If a genotype is not called (that is, a `.` is
+        present in the GT string), -1 is used
+      phaseset (str): If this field is present, this variant call's genotype
+        ordering implies the phase of the bases and is consistent with any other
+        variant calls in the same reference sequence which have the same
+        phaseset value. If the genotype data was phased but no phase set was
+        specified, this field will be set to `*`.
+      info (dict): A map of additional variant call information. The key is
+        specified in the VCF record and the type of the value is specified by
+        the VCF header FORMAT.
+    """
+    self.name = name
+    self.genotype = genotype or []
+    self.phaseset = phaseset
+    self.info = info or {}
+
+  def __eq__(self, other):
+    return ((self.name, self.genotype, self.phaseset, self.info) ==
+            (other.name, other.genotype, other.phaseset, other.info))
+
+  def __repr__(self):
+    return ', '.join(
+        [str(s) for s in [self.name, self.genotype, self.phaseset, self.info]])
+
+
+class _VcfSource(filebasedsource.FileBasedSource):
+  """A source for reading VCF files.
+
+  Parses VCF files (version 4) using PyVCF library. If file_pattern specifies
+  multiple files, then the header from each file is used separately to parse
+  the content. However, the output will be a uniform PCollection of
+  :class:`Variant` objects.
+  """
+
+  DEFAULT_VCF_READ_BUFFER_SIZE = 65536  # 64kB
+
+  def __init__(self,
+               file_pattern,
+               compression_type=CompressionTypes.AUTO,
+               buffer_size=DEFAULT_VCF_READ_BUFFER_SIZE,
+               validate=True):
+    super(_VcfSource, self).__init__(file_pattern,
+                                     compression_type=compression_type,
+                                     validate=validate)
+
+    self._header_lines_per_file = {}
+    self._compression_type = compression_type
+    self._buffer_size = buffer_size
+
+  def read_records(self, file_name, range_tracker):
+    record_iterator = _VcfSource._VcfRecordIterator(
+        file_name,
+        range_tracker,
+        self._pattern,
+        self._compression_type,
+        buffer_size=self._buffer_size,
+        skip_header_lines=0)
+
+    # Convert iterator to generator to abstract behavior
+    for line in record_iterator:
+      yield line
+
+  class _VcfRecordIterator(object):
+    """An Iterator for processing a single VCF file."""
+
+    def __init__(self,
+                 file_name,
+                 range_tracker,
+                 file_pattern,
+                 compression_type,
+                 **kwargs):
+      self._header_lines = []
+      self._last_record = None
+      self._file_name = file_name
+
+      text_source = TextSource(
+          file_pattern,
+          0,  # min_bundle_size
+          compression_type,
+          True,  # strip_trailing_newlines
+          coders.StrUtf8Coder(),  # coder
+          validate=False,
+          header_processor_fns=(lambda x: x.startswith('#'),
+                                self._store_header_lines),
+          **kwargs)
+
+      self._text_lines = text_source.read_records(self._file_name,
+                                                  range_tracker)
+      try:
+        self._vcf_reader = vcf.Reader(fsock=self._create_generator())
+      except SyntaxError as e:
+        raise ValueError('Invalid VCF header %s' % str(e))
+
+    def _store_header_lines(self, header_lines):
+      self._header_lines = header_lines
+
+    def _create_generator(self):
+      header_processed = False
+      for text_line in self._text_lines:
+        if not header_processed and self._header_lines:
+          for header in self._header_lines:
+            self._last_record = header
+            yield self._last_record
+          header_processed = True
+        # PyVCF has explicit str() calls when parsing INFO fields, which fails
+        # with UTF-8 decoded strings. Encode the line back to UTF-8.
+        self._last_record = text_line.encode('utf-8')
+        yield self._last_record
+
+    def __iter__(self):
+      return self
+
+    def next(self):
+      try:
+        record = next(self._vcf_reader)
+        return self._convert_to_variant_record(record, self._vcf_reader.infos,
+                                               self._vcf_reader.formats)
+      except (LookupError, ValueError) as e:
+        raise ValueError('Invalid record in VCF file. Error: %s' % str(e))
+
+    def _convert_to_variant_record(self, record, infos, formats):
+      """Converts the PyVCF record to a :class:`Variant` object.
+
+      Args:
+        record (:class:`~vcf.model._Record`): An object containing info about a
+          variant.
+        infos (dict): The PyVCF dict storing INFO extracted from the VCF header.
+          The key is the info key and the value is :class:`~vcf.parser._Info`.
+        formats (dict): The PyVCF dict storing FORMAT extracted from the VCF
+          header. The key is the FORMAT key and the value is
+          :class:`~vcf.parser._Format`.
+      Returns:
+        A :class:`Variant` object from the given record.
+      """
+      variant = Variant()
+      variant.reference_name = record.CHROM
+      variant.start = record.start
+      variant.end = record.end
+      variant.reference_bases = (
+          record.REF if record.REF != MISSING_FIELD_VALUE else None)
+      # ALT fields are classes in PyVCF (e.g. Substitution), so need convert
+      # them to their string representations.
+      variant.alternate_bases.extend(
+          [str(r) for r in record.ALT if r] if record.ALT else [])
+      variant.names.extend(record.ID.split(';') if record.ID else [])
+      variant.quality = record.QUAL
+      # PyVCF uses None for '.' and an empty list for 'PASS'.
+      if record.FILTER is not None:
+        variant.filters.extend(
+            record.FILTER if record.FILTER else [PASS_FILTER])
+      for k, v in record.INFO.iteritems():
+        # Special case: END info value specifies end of the record, so adjust
+        # variant.end and do not include it as part of variant.info.
+        if k == END_INFO_KEY:
+          variant.end = v
+          continue
+        field_count = None
+        if k in infos:
+          field_count = self._get_field_count_as_string(infos[k].num)
+        variant.info[k] = VariantInfo(data=v, field_count=field_count)
+      for sample in record.samples:
+        call = VariantCall()
+        call.name = sample.sample
+        for allele in sample.gt_alleles or [MISSING_GENOTYPE_VALUE]:
+          if allele is None:
+            allele = MISSING_GENOTYPE_VALUE
+          call.genotype.append(int(allele))
+        phaseset_from_format = (getattr(sample.data, PHASESET_FORMAT_KEY)
+                                if PHASESET_FORMAT_KEY in sample.data._fields
+                                else None)
+        # Note: Call is considered phased if it contains the 'PS' key regardless
+        # of whether it uses '|'.
+        if phaseset_from_format or sample.phased:
+          call.phaseset = (str(phaseset_from_format) if phaseset_from_format
+                           else DEFAULT_PHASESET_VALUE)
+        for field in sample.data._fields:
+          # Genotype and phaseset (if present) are already included.
+          if field in (GENOTYPE_FORMAT_KEY, PHASESET_FORMAT_KEY):
+            continue
+          data = getattr(sample.data, field)
+          # Convert single values to a list for cases where the number of fields
+          # is unknown. This is to ensure consistent types across all records.
+          # Note: this is already done for INFO fields in PyVCF.
+          if (field in formats and
+              formats[field].num is None and
+              isinstance(data, (int, float, long, basestring, bool))):
+            data = [data]
+          call.info[field] = data
+        variant.calls.append(call)
+      return variant
+
+    def _get_field_count_as_string(self, field_count):
+      """Returns the string representation of field_count from PyVCF.
+
+      PyVCF converts field counts to an integer with some predefined constants
+      as specified in the vcf.parser.field_counts dict (e.g. 'A' is -1). This
+      method converts them back to their string representation to avoid having
+      direct dependency on the arbitrary PyVCF constants.
+      Args:
+        field_count (int): An integer representing the number of fields in INFO
+          as specified by PyVCF.
+      Returns:
+        A string representation of field_count (e.g. '-1' becomes 'A').
+      Raises:
+        ValueError: if the field_count is not valid.
+      """
+      if field_count is None:
+        return None
+      elif field_count >= 0:
+        return str(field_count)
+      field_count_to_string = {v: k for k, v in vcf.parser.field_counts.items()}
+      if field_count in field_count_to_string:
+        return field_count_to_string[field_count]
+      else:
+        raise ValueError('Invalid value for field_count: %d' % field_count)
+
+
+class ReadFromVcf(PTransform):
+  """A :class:`~apache_beam.transforms.ptransform.PTransform` for reading VCF
+  files.
+
+  Parses VCF files (version 4) using PyVCF library. If file_pattern specifies
+  multiple files, then the header from each file is used separately to parse
+  the content. However, the output will be a PCollection of
+  :class:`Variant` objects.
+  """
+
+  def __init__(
+      self,
+      file_pattern=None,
+      compression_type=CompressionTypes.AUTO,
+      validate=True,
+      **kwargs):
+    """Initialize the :class:`ReadFromVcf` transform.
+
+    Args:
+      file_pattern (str): The file path to read from either as a single file or
+        a glob pattern.
+      compression_type (str): Used to handle compressed input files.
+        Typical value is :attr:`CompressionTypes.AUTO
+        <apache_beam.io.filesystem.CompressionTypes.AUTO>`, in which case the
+        underlying file_path's extension will be used to detect the compression.
+      validate (bool): flag to verify that the files exist during the pipeline
+        creation time.
+    """
+    super(ReadFromVcf, self).__init__(**kwargs)
+    self._source = _VcfSource(
+        file_pattern, compression_type, validate=validate)
+
+  def expand(self, pvalue):
+    return pvalue.pipeline | Read(self._source)

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/io/vcfio_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/vcfio_test.py b/sdks/python/apache_beam/io/vcfio_test.py
new file mode 100644
index 0000000..871b6e9
--- /dev/null
+++ b/sdks/python/apache_beam/io/vcfio_test.py
@@ -0,0 +1,519 @@
+#
+# 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.
+#
+
+"""Tests for vcfio module."""
+
+import logging
+import os
+import unittest
+from itertools import permutations
+
+import apache_beam.io.source_test_utils as source_test_utils
+from apache_beam.io.vcfio import _VcfSource as VcfSource
+from apache_beam.io.vcfio import DEFAULT_PHASESET_VALUE
+from apache_beam.io.vcfio import MISSING_GENOTYPE_VALUE
+from apache_beam.io.vcfio import ReadFromVcf
+from apache_beam.io.vcfio import Variant
+from apache_beam.io.vcfio import VariantCall
+from apache_beam.io.vcfio import VariantInfo
+from apache_beam.testing.test_pipeline import TestPipeline
+from apache_beam.testing.test_utils import TempDir
+from apache_beam.testing.util import BeamAssertException
+from apache_beam.testing.util import assert_that
+
+# Note: mixing \n and \r\n to verify both behaviors.
+_SAMPLE_HEADER_LINES = [
+    '##fileformat=VCFv4.2\n',
+    '##INFO=<ID=NS,Number=1,Type=Integer,Description="Number samples">\n',
+    '##INFO=<ID=AF,Number=A,Type=Float,Description="Allele Frequency">\n',
+    '##FORMAT=<ID=GT,Number=1,Type=String,Description="Genotype">\r\n',
+    '##FORMAT=<ID=GQ,Number=1,Type=Integer,Description="Genotype Quality">\n',
+    '#CHROM	POS	ID	REF	ALT	QUAL	FILTER	INFO	FORMAT	Sample1	Sample2\r\n',
+]
+
+_SAMPLE_TEXT_LINES = [
+    '20	14370	.	G	A	29	PASS	AF=0.5	GT:GQ	0|0:48 1|0:48\n',
+    '20	17330	.	T	A	3	q10	AF=0.017	GT:GQ	0|0:49	0|1:3\n',
+    '20	1110696	.	A	G,T	67	PASS	AF=0.3,0.7	GT:GQ	1|2:21	2|1:2\n',
+    '20	1230237	.	T	.	47	PASS	.	GT:GQ	0|0:54	0|0:48\n',
+    '19	1234567	.	GTCT	G,GTACT	50	PASS	.	GT:GQ	0/1:35	0/2:17\n',
+    '20	1234	rs123	C	A,T	50	PASS	AF=0.5	GT:GQ	0/0:48	1/0:20\n',
+    '19	123	rs1234	GTC	.	40	q10;s50	NS=2	GT:GQ	1|0:48	0/1:.\n',
+    '19	12	.	C	<SYMBOLIC>	49	q10	AF=0.5	GT:GQ	0|1:45 .:.\n'
+]
+
+
+def get_full_file_path(file_name):
+  """Returns the full path of the specified ``file_name`` from ``data``."""
+  return os.path.join(
+      os.path.dirname(__file__), '..', 'testing', 'data', 'vcf', file_name)
+
+
+def get_full_dir():
+  """Returns the full path of the  ``data`` directory."""
+  return os.path.join(os.path.dirname(__file__), '..', 'testing', 'data', 'vcf')
+
+
+# Helper method for comparing variants.
+def _variant_comparator(v1, v2):
+  if v1.reference_name == v2.reference_name:
+    if v1.start == v2.start:
+      return cmp(v1.end, v2.end)
+    return cmp(v1.start, v2.start)
+  return cmp(v1.reference_name, v2.reference_name)
+
+
+# Helper method for verifying equal count on PCollection.
+def _count_equals_to(expected_count):
+  def _count_equal(actual_list):
+    actual_count = len(actual_list)
+    if expected_count != actual_count:
+      raise BeamAssertException(
+          'Expected %d not equal actual %d' % (expected_count, actual_count))
+  return _count_equal
+
+
+class VcfSourceTest(unittest.TestCase):
+
+  # Distribution should skip tests that need VCF files due to large size
+  VCF_FILE_DIR_MISSING = not os.path.exists(get_full_dir())
+
+  def _create_temp_vcf_file(self, lines, tempdir):
+    return tempdir.create_temp_file(suffix='.vcf', lines=lines)
+
+  def _read_records(self, file_or_pattern):
+    return source_test_utils.read_from_source(VcfSource(file_or_pattern))
+
+  def _create_temp_file_and_read_records(self, lines):
+    with TempDir() as tempdir:
+      file_name = tempdir.create_temp_file(suffix='.vcf', lines=lines)
+      return self._read_records(file_name)
+
+  def _assert_variants_equal(self, actual, expected):
+    self.assertEqual(
+        sorted(expected),
+        sorted(actual))
+
+  def _get_sample_variant_1(self):
+    """Get first sample variant.
+
+    Features:
+      multiple alternates
+      not phased
+      multiple names
+    """
+    vcf_line = ('20	1234	rs123;rs2	C	A,T	50	PASS	AF=0.5,0.1;NS=1	'
+                'GT:GQ	0/0:48	1/0:20\n')
+    variant = Variant(
+        reference_name='20', start=1233, end=1234, reference_bases='C',
+        alternate_bases=['A', 'T'], names=['rs123', 'rs2'], quality=50,
+        filters=['PASS'],
+        info={'AF': VariantInfo(data=[0.5, 0.1], field_count='A'),
+              'NS': VariantInfo(data=1, field_count='1')})
+    variant.calls.append(
+        VariantCall(name='Sample1', genotype=[0, 0], info={'GQ': 48}))
+    variant.calls.append(
+        VariantCall(name='Sample2', genotype=[1, 0], info={'GQ': 20}))
+    return variant, vcf_line
+
+  def _get_sample_variant_2(self):
+    """Get second sample variant.
+
+    Features:
+      multiple references
+      no alternate
+      phased
+      multiple filters
+      missing format field
+    """
+    vcf_line = (
+        '19	123	rs1234	GTC	.	40	q10;s50	NS=2	GT:GQ	1|0:48	0/1:.\n')
+    variant = Variant(
+        reference_name='19', start=122, end=125, reference_bases='GTC',
+        alternate_bases=[], names=['rs1234'], quality=40,
+        filters=['q10', 's50'],
+        info={'NS': VariantInfo(data=2, field_count='1')})
+    variant.calls.append(
+        VariantCall(name='Sample1', genotype=[1, 0],
+                    phaseset=DEFAULT_PHASESET_VALUE,
+                    info={'GQ': 48}))
+    variant.calls.append(
+        VariantCall(name='Sample2', genotype=[0, 1], info={'GQ': None}))
+    return variant, vcf_line
+
+  def _get_sample_variant_3(self):
+    """Get third sample variant.
+
+    Features:
+      symbolic alternate
+      no calls for sample 2
+    """
+    vcf_line = (
+        '19	12	.	C	<SYMBOLIC>	49	q10	AF=0.5	GT:GQ	0|1:45 .:.\n')
+    variant = Variant(
+        reference_name='19', start=11, end=12, reference_bases='C',
+        alternate_bases=['<SYMBOLIC>'], quality=49, filters=['q10'],
+        info={'AF': VariantInfo(data=[0.5], field_count='A')})
+    variant.calls.append(
+        VariantCall(name='Sample1', genotype=[0, 1],
+                    phaseset=DEFAULT_PHASESET_VALUE,
+                    info={'GQ': 45}))
+    variant.calls.append(
+        VariantCall(name='Sample2', genotype=[MISSING_GENOTYPE_VALUE],
+                    info={'GQ': None}))
+    return variant, vcf_line
+
+  def test_sort_variants(self):
+    sorted_variants = [
+        Variant(reference_name='a', start=20, end=22),
+        Variant(reference_name='a', start=20, end=22, quality=20),
+        Variant(reference_name='b', start=20, end=22),
+        Variant(reference_name='b', start=21, end=22),
+        Variant(reference_name='b', start=21, end=23)]
+
+    for permutation in permutations(sorted_variants):
+      self.assertEqual(sorted(permutation), sorted_variants)
+
+  def test_variant_equality(self):
+    base_variant = Variant(reference_name='a', start=20, end=22,
+                           reference_bases='a', alternate_bases=['g', 't'],
+                           names=['variant'], quality=9, filters=['q10'],
+                           info={'key': 'value'},
+                           calls=[VariantCall(genotype=[0, 0])])
+    equal_variant = Variant(reference_name='a', start=20, end=22,
+                            reference_bases='a', alternate_bases=['g', 't'],
+                            names=['variant'], quality=9, filters=['q10'],
+                            info={'key': 'value'},
+                            calls=[VariantCall(genotype=[0, 0])])
+    different_calls = Variant(reference_name='a', start=20, end=22,
+                              reference_bases='a', alternate_bases=['g', 't'],
+                              names=['variant'], quality=9, filters=['q10'],
+                              info={'key': 'value'},
+                              calls=[VariantCall(genotype=[1, 0])])
+    missing_field = Variant(reference_name='a', start=20, end=22,
+                            reference_bases='a', alternate_bases=['g', 't'],
+                            names=['variant'], quality=9, filters=['q10'],
+                            info={'key': 'value'})
+
+    self.assertEqual(base_variant, equal_variant)
+    self.assertNotEqual(base_variant, different_calls)
+    self.assertNotEqual(base_variant, missing_field)
+
+  @unittest.skipIf(VCF_FILE_DIR_MISSING, 'VCF test file directory is missing')
+  def test_read_single_file_large(self):
+    test_data_conifgs = [
+        {'file': 'valid-4.0.vcf', 'num_records': 5},
+        {'file': 'valid-4.0.vcf.gz', 'num_records': 5},
+        {'file': 'valid-4.0.vcf.bz2', 'num_records': 5},
+        {'file': 'valid-4.1-large.vcf', 'num_records': 9882},
+        {'file': 'valid-4.2.vcf', 'num_records': 13},
+    ]
+    for config in test_data_conifgs:
+      read_data = self._read_records(
+          get_full_file_path(config['file']))
+      self.assertEqual(config['num_records'], len(read_data))
+
+  @unittest.skipIf(VCF_FILE_DIR_MISSING, 'VCF test file directory is missing')
+  def test_read_file_pattern_large(self):
+    read_data = self._read_records(
+        os.path.join(get_full_dir(), 'valid-*.vcf'))
+    self.assertEqual(9900, len(read_data))
+    read_data_gz = self._read_records(
+        os.path.join(get_full_dir(), 'valid-*.vcf.gz'))
+    self.assertEqual(9900, len(read_data_gz))
+
+  def test_single_file_no_records(self):
+    self.assertEqual(
+        [], self._create_temp_file_and_read_records(['']))
+    self.assertEqual(
+        [], self._create_temp_file_and_read_records(['\n', '\r\n', '\n']))
+    self.assertEqual(
+        [], self._create_temp_file_and_read_records(_SAMPLE_HEADER_LINES))
+
+  def test_single_file_verify_details(self):
+    variant_1, vcf_line_1 = self._get_sample_variant_1()
+    read_data = self._create_temp_file_and_read_records(
+        _SAMPLE_HEADER_LINES + [vcf_line_1])
+    self.assertEqual(1, len(read_data))
+    self.assertEqual(variant_1, read_data[0])
+    variant_2, vcf_line_2 = self._get_sample_variant_2()
+    variant_3, vcf_line_3 = self._get_sample_variant_3()
+    read_data = self._create_temp_file_and_read_records(
+        _SAMPLE_HEADER_LINES + [vcf_line_1, vcf_line_2, vcf_line_3])
+    self.assertEqual(3, len(read_data))
+    self._assert_variants_equal([variant_1, variant_2, variant_3], read_data)
+
+  def test_file_pattern_verify_details(self):
+    variant_1, vcf_line_1 = self._get_sample_variant_1()
+    variant_2, vcf_line_2 = self._get_sample_variant_2()
+    variant_3, vcf_line_3 = self._get_sample_variant_3()
+    with TempDir() as tempdir:
+      self._create_temp_vcf_file(_SAMPLE_HEADER_LINES + [vcf_line_1], tempdir)
+      self._create_temp_vcf_file((_SAMPLE_HEADER_LINES +
+                                  [vcf_line_2, vcf_line_3]),
+                                 tempdir)
+      read_data = self._read_records(os.path.join(tempdir.get_path(), '*.vcf'))
+      self.assertEqual(3, len(read_data))
+      self._assert_variants_equal([variant_1, variant_2, variant_3], read_data)
+
+  @unittest.skipIf(VCF_FILE_DIR_MISSING, 'VCF test file directory is missing')
+  def test_read_after_splitting(self):
+    file_name = get_full_file_path('valid-4.1-large.vcf')
+    source = VcfSource(file_name)
+    splits = [p for p in source.split(desired_bundle_size=500)]
+    self.assertGreater(len(splits), 1)
+    sources_info = ([
+        (split.source, split.start_position, split.stop_position) for
+        split in splits])
+    self.assertGreater(len(sources_info), 1)
+    split_records = []
+    for source_info in sources_info:
+      split_records.extend(source_test_utils.read_from_source(*source_info))
+    self.assertEqual(9882, len(split_records))
+
+  def test_invalid_file(self):
+    invalid_file_contents = [
+        # Malfromed record.
+        [
+            '#CHROM	POS	ID	REF	ALT	QUAL	FILTER	INFO	FORMAT	SampleName\n',
+            '1    1  '
+        ],
+        # Missing "GT:GQ" format, but GQ is provided.
+        [
+            '#CHROM	POS	ID	REF	ALT	QUAL	FILTER	INFO	FORMAT	SampleName\n',
+            '19	123	rs12345	T	C	50	q10	AF=0.2;NS=2	GT	1|0:48'
+        ],
+        # GT is not an integer.
+        [
+            '#CHROM	POS	ID	REF	ALT	QUAL	FILTER	INFO	FORMAT	SampleName\n',
+            '19	123	rs12345	T	C	50	q10	AF=0.2;NS=2	GT	A|0'
+        ],
+        # Malformed FILTER.
+        [
+            '##FILTER=<ID=PASS,Description="All filters passed">\n',
+            '##FILTER=<ID=LowQual,Descri\n',
+            '#CHROM	POS	ID	REF	ALT	QUAL	FILTER	INFO	FORMAT	SampleName\n',
+            '19	123	rs12345	T	C	50	q10	AF=0.2;NS=2	GT:GQ	1|0:48',
+        ],
+        # Invalid Number value for INFO.
+        [
+            '##INFO=<ID=G,Number=U,Type=String,Description="InvalidNumber">\n',
+            '#CHROM	POS	ID	REF	ALT	QUAL	FILTER	INFO	FORMAT	SampleName\n',
+            '19	123	rs12345	T	C	50	q10	AF=0.2;NS=2	GT:GQ	1|0:48\n',
+        ],
+        # POS should be an integer.
+        [
+            '##FILTER=<ID=PASS,Description="All filters passed">\n',
+            '##FILTER=<ID=LowQual,Descri\n',
+            '#CHROM	POS	ID	REF	ALT	QUAL	FILTER	INFO	FORMAT	SampleName\n',
+            '19	abc	rs12345	T	C	50	q10	AF=0.2;NS=2	GT:GQ	1|0:48\n',
+        ],
+    ]
+    for content in invalid_file_contents:
+      try:
+        with TempDir() as tempdir:
+          self._read_records(self._create_temp_vcf_file(content, tempdir))
+          self.fail('Invalid VCF file must throw an exception')
+      except ValueError:
+        pass
+    # Try with multiple files (any one of them will throw an exception).
+    with TempDir() as tempdir:
+      for content in invalid_file_contents:
+        self._create_temp_vcf_file(content, tempdir)
+      try:
+        self._read_records(os.path.join(tempdir.get_path(), '*.vcf'))
+        self.fail('Invalid VCF file must throw an exception.')
+      except ValueError:
+        pass
+
+  def test_no_samples(self):
+    header_line = '#CHROM	POS	ID	REF	ALT	QUAL	FILTER	INFO\n'
+    record_line = '19	123	.	G	A	.	PASS	AF=0.2'
+    expected_variant = Variant(
+        reference_name='19', start=122, end=123, reference_bases='G',
+        alternate_bases=['A'], filters=['PASS'],
+        info={'AF': VariantInfo(data=[0.2], field_count='A')})
+    read_data = self._create_temp_file_and_read_records(
+        _SAMPLE_HEADER_LINES[:-1] + [header_line, record_line])
+    self.assertEqual(1, len(read_data))
+    self.assertEqual(expected_variant, read_data[0])
+
+  def test_no_info(self):
+    record_line = 'chr19	123	.	.	.	.	.	.	GT	.	.'
+    expected_variant = Variant(reference_name='chr19', start=122, end=123)
+    expected_variant.calls.append(
+        VariantCall(name='Sample1', genotype=[MISSING_GENOTYPE_VALUE]))
+    expected_variant.calls.append(
+        VariantCall(name='Sample2', genotype=[MISSING_GENOTYPE_VALUE]))
+    read_data = self._create_temp_file_and_read_records(
+        _SAMPLE_HEADER_LINES + [record_line])
+    self.assertEqual(1, len(read_data))
+    self.assertEqual(expected_variant, read_data[0])
+
+  def test_info_numbers_and_types(self):
+    info_headers = [
+        '##INFO=<ID=HA,Number=A,Type=String,Description="StringInfo_A">\n',
+        '##INFO=<ID=HG,Number=G,Type=Integer,Description="IntInfo_G">\n',
+        '##INFO=<ID=HR,Number=R,Type=Character,Description="ChrInfo_R">\n',
+        '##INFO=<ID=HF,Number=0,Type=Flag,Description="FlagInfo">\n',
+        '##INFO=<ID=HU,Number=.,Type=Float,Description="FloatInfo_variable">\n']
+    record_lines = [
+        '19	2	.	A	T,C	.	.	HA=a1,a2;HG=1,2,3;HR=a,b,c;HF;HU=0.1	GT	1/0	0/1\n',
+        '19	124	.	A	T	.	.	HG=3,4,5;HR=d,e;HU=1.1,1.2	GT	0/0	0/1']
+    variant_1 = Variant(
+        reference_name='19', start=1, end=2, reference_bases='A',
+        alternate_bases=['T', 'C'],
+        info={'HA': VariantInfo(data=['a1', 'a2'], field_count='A'),
+              'HG': VariantInfo(data=[1, 2, 3], field_count='G'),
+              'HR': VariantInfo(data=['a', 'b', 'c'], field_count='R'),
+              'HF': VariantInfo(data=True, field_count='0'),
+              'HU': VariantInfo(data=[0.1], field_count=None)})
+    variant_1.calls.append(VariantCall(name='Sample1', genotype=[1, 0]))
+    variant_1.calls.append(VariantCall(name='Sample2', genotype=[0, 1]))
+    variant_2 = Variant(
+        reference_name='19', start=123, end=124, reference_bases='A',
+        alternate_bases=['T'],
+        info={'HG': VariantInfo(data=[3, 4, 5], field_count='G'),
+              'HR': VariantInfo(data=['d', 'e'], field_count='R'),
+              'HU': VariantInfo(data=[1.1, 1.2], field_count=None)})
+    variant_2.calls.append(VariantCall(name='Sample1', genotype=[0, 0]))
+    variant_2.calls.append(VariantCall(name='Sample2', genotype=[0, 1]))
+    read_data = self._create_temp_file_and_read_records(
+        info_headers + _SAMPLE_HEADER_LINES[1:] + record_lines)
+    self.assertEqual(2, len(read_data))
+    self._assert_variants_equal([variant_1, variant_2], read_data)
+
+  def test_end_info_key(self):
+    phaseset_header_line = (
+        '##INFO=<ID=END,Number=1,Type=Integer,Description="End of record.">\n')
+    record_lines = ['19	123	.	A	.	.	.	END=1111	GT	1/0	0/1\n',
+                    '19	123	.	A	.	.	.	.	GT	0/1	1/1\n']
+    variant_1 = Variant(
+        reference_name='19', start=122, end=1111, reference_bases='A')
+    variant_1.calls.append(VariantCall(name='Sample1', genotype=[1, 0]))
+    variant_1.calls.append(VariantCall(name='Sample2', genotype=[0, 1]))
+    variant_2 = Variant(
+        reference_name='19', start=122, end=123, reference_bases='A')
+    variant_2.calls.append(VariantCall(name='Sample1', genotype=[0, 1]))
+    variant_2.calls.append(VariantCall(name='Sample2', genotype=[1, 1]))
+    read_data = self._create_temp_file_and_read_records(
+        [phaseset_header_line] + _SAMPLE_HEADER_LINES[1:] + record_lines)
+    self.assertEqual(2, len(read_data))
+    self._assert_variants_equal([variant_1, variant_2], read_data)
+
+  def test_custom_phaseset(self):
+    phaseset_header_line = (
+        '##FORMAT=<ID=PS,Number=1,Type=Integer,Description="Phaseset">\n')
+    record_lines = ['19	123	.	A	T	.	.	.	GT:PS	1|0:1111	0/1:.\n',
+                    '19	121	.	A	T	.	.	.	GT:PS	1|0:2222	0/1:2222\n']
+    variant_1 = Variant(
+        reference_name='19', start=122, end=123, reference_bases='A',
+        alternate_bases=['T'])
+    variant_1.calls.append(
+        VariantCall(name='Sample1', genotype=[1, 0], phaseset='1111'))
+    variant_1.calls.append(VariantCall(name='Sample2', genotype=[0, 1]))
+    variant_2 = Variant(
+        reference_name='19', start=120, end=121, reference_bases='A',
+        alternate_bases=['T'])
+    variant_2.calls.append(
+        VariantCall(name='Sample1', genotype=[1, 0], phaseset='2222'))
+    variant_2.calls.append(
+        VariantCall(name='Sample2', genotype=[0, 1], phaseset='2222'))
+    read_data = self._create_temp_file_and_read_records(
+        [phaseset_header_line] + _SAMPLE_HEADER_LINES[1:] + record_lines)
+    self.assertEqual(2, len(read_data))
+    self._assert_variants_equal([variant_1, variant_2], read_data)
+
+  def test_format_numbers(self):
+    format_headers = [
+        '##FORMAT=<ID=FU,Number=.,Type=String,Description="Format_variable">\n',
+        '##FORMAT=<ID=F1,Number=1,Type=Integer,Description="Format_one">\n',
+        '##FORMAT=<ID=F2,Number=2,Type=Character,Description="Format_two">\n']
+    record_lines = [
+        '19	2	.	A	T,C	.	.	.	GT:FU:F1:F2	1/0:a1:3:a,b	0/1:a2,a3:4:b,c\n']
+    expected_variant = Variant(
+        reference_name='19', start=1, end=2, reference_bases='A',
+        alternate_bases=['T', 'C'])
+    expected_variant.calls.append(VariantCall(
+        name='Sample1',
+        genotype=[1, 0],
+        info={'FU': ['a1'], 'F1': 3, 'F2': ['a', 'b']}))
+    expected_variant.calls.append(VariantCall(
+        name='Sample2',
+        genotype=[0, 1],
+        info={'FU': ['a2', 'a3'], 'F1': 4, 'F2': ['b', 'c']}))
+    read_data = self._create_temp_file_and_read_records(
+        format_headers + _SAMPLE_HEADER_LINES[1:] + record_lines)
+    self.assertEqual(1, len(read_data))
+    self.assertEqual(expected_variant, read_data[0])
+
+  def test_pipeline_read_single_file(self):
+    with TempDir() as tempdir:
+      file_name = self._create_temp_vcf_file(_SAMPLE_HEADER_LINES +
+                                             _SAMPLE_TEXT_LINES, tempdir)
+      pipeline = TestPipeline()
+      pcoll = pipeline | 'Read' >> ReadFromVcf(file_name)
+      assert_that(pcoll, _count_equals_to(len(_SAMPLE_TEXT_LINES)))
+      pipeline.run()
+
+  @unittest.skipIf(VCF_FILE_DIR_MISSING, 'VCF test file directory is missing')
+  def test_pipeline_read_single_file_large(self):
+    pipeline = TestPipeline()
+    pcoll = pipeline | 'Read' >> ReadFromVcf(
+        get_full_file_path('valid-4.0.vcf'))
+    assert_that(pcoll, _count_equals_to(5))
+    pipeline.run()
+
+  @unittest.skipIf(VCF_FILE_DIR_MISSING, 'VCF test file directory is missing')
+  def test_pipeline_read_file_pattern_large(self):
+    pipeline = TestPipeline()
+    pcoll = pipeline | 'Read' >> ReadFromVcf(
+        os.path.join(get_full_dir(), 'valid-*.vcf'))
+    assert_that(pcoll, _count_equals_to(9900))
+    pipeline.run()
+
+  def test_read_reentrant_without_splitting(self):
+    with TempDir() as tempdir:
+      file_name = self._create_temp_vcf_file(_SAMPLE_HEADER_LINES +
+                                             _SAMPLE_TEXT_LINES, tempdir)
+      source = VcfSource(file_name)
+      source_test_utils.assert_reentrant_reads_succeed((source, None, None))
+
+  def test_read_reentrant_after_splitting(self):
+    with TempDir() as tempdir:
+      file_name = self._create_temp_vcf_file(_SAMPLE_HEADER_LINES +
+                                             _SAMPLE_TEXT_LINES, tempdir)
+      source = VcfSource(file_name)
+      splits = [split for split in source.split(desired_bundle_size=100000)]
+      assert len(splits) == 1
+      source_test_utils.assert_reentrant_reads_succeed(
+          (splits[0].source, splits[0].start_position, splits[0].stop_position))
+
+  def test_dynamic_work_rebalancing(self):
+    with TempDir() as tempdir:
+      file_name = self._create_temp_vcf_file(_SAMPLE_HEADER_LINES +
+                                             _SAMPLE_TEXT_LINES, tempdir)
+      source = VcfSource(file_name)
+      splits = [split for split in source.split(desired_bundle_size=100000)]
+      assert len(splits) == 1
+      source_test_utils.assert_split_at_fraction_exhaustive(
+          splits[0].source, splits[0].start_position, splits[0].stop_position)
+
+
+if __name__ == '__main__':
+  logging.getLogger().setLevel(logging.INFO)
+  unittest.main()

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/testing/data/vcf/valid-4.0.vcf
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/testing/data/vcf/valid-4.0.vcf b/sdks/python/apache_beam/testing/data/vcf/valid-4.0.vcf
new file mode 100644
index 0000000..e9b064b
--- /dev/null
+++ b/sdks/python/apache_beam/testing/data/vcf/valid-4.0.vcf
@@ -0,0 +1,23 @@
+##fileformat=VCFv4.0
+##fileDate=20090805
+##source=myImputationProgramV3.1
+##reference=1000GenomesPilot-NCBI36
+##phasing=partial
+##INFO=<ID=NS,Number=1,Type=Integer,Description="Number of Samples With Data">
+##INFO=<ID=DP,Number=1,Type=Integer,Description="Total Depth">
+##INFO=<ID=AF,Number=.,Type=Float,Description="Allele Frequency">
+##INFO=<ID=AA,Number=1,Type=String,Description="Ancestral Allele">
+##INFO=<ID=DB,Number=0,Type=Flag,Description="dbSNP membership, build 129">
+##INFO=<ID=H2,Number=0,Type=Flag,Description="HapMap2 membership">
+##FILTER=<ID=q10,Description="Quality below 10">
+##FILTER=<ID=s50,Description="Less than 50% of samples have data">
+##FORMAT=<ID=GT,Number=1,Type=String,Description="Genotype">
+##FORMAT=<ID=GQ,Number=1,Type=Integer,Description="Genotype Quality">
+##FORMAT=<ID=DP,Number=1,Type=Integer,Description="Read Depth">
+##FORMAT=<ID=HQ,Number=2,Type=Integer,Description="Haplotype Quality">
+#CHROM	POS	ID	REF	ALT	QUAL	FILTER	INFO	FORMAT	NA00001	NA00002	NA00003
+20	14370	rs6054257	G	A	29	PASS	NS=3;DP=14;AF=0.5;DB;H2	GT:GQ:DP:HQ	0|0:48:1:51,51	1|0:48:8:51,51	1/1:43:5:.,.
+20	17330	.	T	A	3	q10	NS=3;DP=11;AF=0.017	GT:GQ:DP:HQ	0|0:49:3:58,50	0|1:3:5:65,3	0/0:41:3
+20	1110696	rs6040355	A	G,T	67	PASS	NS=2;DP=10;AF=0.333,0.667;AA=T;DB	GT:GQ:DP:HQ	1|2:21:6:23,27	2|1:2:0:18,2	2/2:35:4
+20	1230237	.	T	.	47	PASS	NS=3;DP=13;AA=T	GT:GQ:DP:HQ	0|0:54:7:56,60	0|0:48:4:51,51	0/0:61:2
+19	1234567	microsat1	GTCT	G,GTACT	50	PASS	NS=3;DP=9;AA=G	GT:GQ:DP	0/1:35:4	0/2:17:2	1/1:40:3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/testing/data/vcf/valid-4.0.vcf.bz2
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/testing/data/vcf/valid-4.0.vcf.bz2 b/sdks/python/apache_beam/testing/data/vcf/valid-4.0.vcf.bz2
new file mode 100644
index 0000000..dff64e6
Binary files /dev/null and b/sdks/python/apache_beam/testing/data/vcf/valid-4.0.vcf.bz2 differ

http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/testing/data/vcf/valid-4.0.vcf.gz
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/testing/data/vcf/valid-4.0.vcf.gz b/sdks/python/apache_beam/testing/data/vcf/valid-4.0.vcf.gz
new file mode 100644
index 0000000..ac58128
Binary files /dev/null and b/sdks/python/apache_beam/testing/data/vcf/valid-4.0.vcf.gz differ


[2/4] beam git commit: Added vcf file io source and modified _TextSource to optionally handle headers

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/testing/data/vcf/valid-4.1-large.vcf
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/testing/data/vcf/valid-4.1-large.vcf b/sdks/python/apache_beam/testing/data/vcf/valid-4.1-large.vcf
new file mode 100644
index 0000000..c470685
--- /dev/null
+++ b/sdks/python/apache_beam/testing/data/vcf/valid-4.1-large.vcf
@@ -0,0 +1,10000 @@
+##fileformat=VCFv4.1
+##fileDate=20121204
+##center=Complete Genomics
+##source=CGAPipeline_2.2.0.26
+##source_GENOME_REFERENCE=NCBI build 37
+##source_MAX_PLOIDY=10
+##source_NUMBER_LEVELS=GS01868-DNA_H02:7
+##source_NONDIPLOID_WINDOW_WIDTH=100000
+##source_MEAN_GC_CORRECTED_CVG=GS01868-DNA_H02:41.51
+##source_GENE_ANNOTATIONS=NCBI build 37.2
+##source_DBSNP_BUILD=dbSNP build 135
+##source_MEI_1000G_ANNOTATIONS=INITIAL-DATA-RELEASE
+##source_COSMIC=COSMIC v59
+##source_DGV_VERSION=9
+##source_MIRBASE_VERSION=mirBase version 18
+##source_PFAM_DATE=April 21, 2011
+##source_REPMASK_GENERATED_AT=2011-Feb-15 10:08
+##source_SEGDUP_GENERATED_AT=2010-Dec-01 13:40
+##phasing=partial
+##reference=ftp://ftp.completegenomics.com/ReferenceFiles/build37.fa.bz2
+##contig=<ID=1,length=249250621,assembly=B37,md5=1b22b98cdeb4a9304cb5d48026a85128,species="Homo sapiens">
+##contig=<ID=2,length=243199373,assembly=B37,md5=a0d9851da00400dec1098a9255ac712e,species="Homo sapiens">
+##contig=<ID=3,length=198022430,assembly=B37,md5=641e4338fa8d52a5b781bd2a2c08d3c3,species="Homo sapiens">
+##contig=<ID=4,length=191154276,assembly=B37,md5=23dccd106897542ad87d2765d28a19a1,species="Homo sapiens">
+##contig=<ID=5,length=180915260,assembly=B37,md5=0740173db9ffd264d728f32784845cd7,species="Homo sapiens">
+##contig=<ID=6,length=171115067,assembly=B37,md5=1d3a93a248d92a729ee764823acbbc6b,species="Homo sapiens">
+##contig=<ID=7,length=159138663,assembly=B37,md5=618366e953d6aaad97dbe4777c29375e,species="Homo sapiens">
+##contig=<ID=8,length=146364022,assembly=B37,md5=96f514a9929e410c6651697bded59aec,species="Homo sapiens">
+##contig=<ID=9,length=141213431,assembly=B37,md5=3e273117f15e0a400f01055d9f393768,species="Homo sapiens">
+##contig=<ID=10,length=135534747,assembly=B37,md5=988c28e000e84c26d552359af1ea2e1d,species="Homo sapiens">
+##contig=<ID=11,length=135006516,assembly=B37,md5=98c59049a2df285c76ffb1c6db8f8b96,species="Homo sapiens">
+##contig=<ID=12,length=133851895,assembly=B37,md5=51851ac0e1a115847ad36449b0015864,species="Homo sapiens">
+##contig=<ID=13,length=115169878,assembly=B37,md5=283f8d7892baa81b510a015719ca7b0b,species="Homo sapiens">
+##contig=<ID=14,length=107349540,assembly=B37,md5=98f3cae32b2a2e9524bc19813927542e,species="Homo sapiens">
+##contig=<ID=15,length=102531392,assembly=B37,md5=e5645a794a8238215b2cd77acb95a078,species="Homo sapiens">
+##contig=<ID=16,length=90354753,assembly=B37,md5=fc9b1a7b42b97a864f56b348b06095e6,species="Homo sapiens">
+##contig=<ID=17,length=81195210,assembly=B37,md5=351f64d4f4f9ddd45b35336ad97aa6de,species="Homo sapiens">
+##contig=<ID=18,length=78077248,assembly=B37,md5=b15d4b2d29dde9d3e4f93d1d0f2cbc9c,species="Homo sapiens">
+##contig=<ID=19,length=59128983,assembly=B37,md5=1aacd71f30db8e561810913e0b72636d,species="Homo sapiens">
+##contig=<ID=20,length=63025520,assembly=B37,md5=0dec9660ec1efaaf33281c0d5ea2560f,species="Homo sapiens">
+##contig=<ID=21,length=48129895,assembly=B37,md5=2979a6085bfe28e3ad6f552f361ed74d,species="Homo sapiens">
+##contig=<ID=22,length=51304566,assembly=B37,md5=a718acaa6135fdca8357d5bfe94211dd,species="Homo sapiens">
+##contig=<ID=X,length=155270560,assembly=B37,md5=7e0e2e580297b7764e31dbc80c2540dd,species="Homo sapiens">
+##contig=<ID=Y,length=59373566,assembly=B37,md5=1e86411d73e6f00a10590f976be01623,species="Homo sapiens">
+##contig=<ID=M,length=16569,assembly=B37,md5=c68f52674c9fb33aef52dcf399755519,species="Homo sapiens">
+##ALT=<ID=CGA_CNVWIN,Description="Copy number analysis window">
+##ALT=<ID=INS:ME:ALU,Description="Insertion of ALU element">
+##ALT=<ID=INS:ME:L1,Description="Insertion of L1 element">
+##ALT=<ID=INS:ME:SVA,Description="Insertion of SVA element">
+##ALT=<ID=INS:ME:MER,Description="Insertion of MER element">
+##ALT=<ID=INS:ME:LTR,Description="Insertion of LTR element">
+##ALT=<ID=INS:ME:PolyA,Description="Insertion of PolyA element">
+##ALT=<ID=INS:ME:HERV,Description="Insertion of HERV element">
+##ALT=<ID=CGA_NOCALL,Description="No-called record">
+##FILTER=<ID=URR,Description="Too close to an underrepresented repeat">
+##FILTER=<ID=MPCBT,Description="Mate pair count below 10">
+##FILTER=<ID=SHORT,Description="Junction side length below 70">
+##FILTER=<ID=TSNR,Description="Transition sequence not resolved">
+##FILTER=<ID=INTERBL,Description="Interchromosomal junction in baseline">
+##FILTER=<ID=sns75,Description="Sensitivity to known MEI calls in range (.75,.95] i.e. medium FDR">
+##FILTER=<ID=sns95,Description="Sensitivity to known MEI calls in range (.95,1.00] i.e. high to very high FDR">
+##FILTER=<ID=VQLOW,Description="Quality not VQHIGH">
+##FILTER=<ID=SQLOW,Description="Somatic quality not SQHIGH">
+##INFO=<ID=NS,Number=1,Type=Integer,Description="Number of Samples With Data">
+##INFO=<ID=CGA_WINEND,Number=1,Type=Integer,Description="End of coverage window">
+##INFO=<ID=CGA_BF,Number=1,Type=Float,Description="Frequency in baseline">
+##INFO=<ID=CGA_MEDEL,Number=4,Type=String,Description="Consistent with deletion of mobile element; type,chromosome,start,end">
+##INFO=<ID=CGA_XR,Number=A,Type=String,Description="Per-ALT external database reference (dbSNP, COSMIC, etc)">
+##INFO=<ID=MATEID,Number=1,Type=String,Description="ID of mate breakend">
+##INFO=<ID=SVTYPE,Number=1,Type=String,Description="Type of structural variant">
+##INFO=<ID=CGA_BNDG,Number=A,Type=String,Description="Transcript name and strand of genes containing breakend">
+##INFO=<ID=CGA_BNDGO,Number=A,Type=String,Description="Transcript name and strand of genes containing mate breakend">
+##INFO=<ID=CIPOS,Number=2,Type=Integer,Description="Confidence interval around POS for imprecise variants">
+##INFO=<ID=END,Number=1,Type=Integer,Description="End position of the variant described in this record">
+##INFO=<ID=IMPRECISE,Number=0,Type=Flag,Description="Imprecise structural variation">
+##INFO=<ID=MEINFO,Number=4,Type=String,Description="Mobile element info of the form NAME,START,END,POLARITY">
+##INFO=<ID=SVLEN,Number=.,Type=Integer,Description="Difference in length between REF and ALT alleles">
+##INFO=<ID=AN,Number=1,Type=Integer,Description="Total number of alleles in called genotypes">
+##INFO=<ID=AC,Number=A,Type=Integer,Description="Allele count in genotypes, for each ALT allele">
+##INFO=<ID=CGA_FI,Number=A,Type=String,Description="Functional impact annotation">
+##INFO=<ID=CGA_PFAM,Number=.,Type=String,Description="PFAM Domain">
+##INFO=<ID=CGA_MIRB,Number=.,Type=String,Description="miRBaseId">
+##INFO=<ID=CGA_RPT,Number=.,Type=String,Description="repeatMasker overlap information">
+##INFO=<ID=CGA_SDO,Number=1,Type=Integer,Description="Number of distinct segmental duplications that overlap this locus">
+##FORMAT=<ID=GT,Number=1,Type=String,Description="Genotype">
+##FORMAT=<ID=CGA_GP,Number=1,Type=Float,Description="Depth of coverage for 2k window GC normalized to mean">
+##FORMAT=<ID=CGA_NP,Number=1,Type=Float,Description="Coverage for 2k window, GC-corrected and normalized relative to copy-number-corrected multi-sample baseline">
+##FORMAT=<ID=CGA_CL,Number=1,Type=Float,Description="Nondiploid-model called level">
+##FORMAT=<ID=CGA_LS,Number=1,Type=Integer,Description="Nondiploid-model called level score">
+##FORMAT=<ID=CGA_CP,Number=1,Type=Integer,Description="Diploid-model called ploidy">
+##FORMAT=<ID=CGA_PS,Number=1,Type=Integer,Description="Diploid-model called ploidy score">
+##FORMAT=<ID=CGA_CT,Number=1,Type=String,Description="Diploid-model CNV type">
+##FORMAT=<ID=CGA_TS,Number=1,Type=Integer,Description="Diploid-model CNV type score">
+##FORMAT=<ID=FT,Number=1,Type=String,Description="Genotype filters">
+##FORMAT=<ID=CGA_BNDMPC,Number=1,Type=Integer,Description="Mate pair count supporting breakend">
+##FORMAT=<ID=CGA_BNDPOS,Number=1,Type=Integer,Description="Breakend position">
+##FORMAT=<ID=CGA_BNDDEF,Number=1,Type=String,Description="Breakend definition">
+##FORMAT=<ID=CGA_BNDP,Number=1,Type=String,Description="Precision of breakend">
+##FORMAT=<ID=CGA_IS,Number=1,Type=Float,Description="MEI InsertionScore: confidence in occurrence of an insertion">
+##FORMAT=<ID=CGA_IDC,Number=1,Type=Float,Description="MEI InsertionDnbCount: count of paired ends supporting insertion">
+##FORMAT=<ID=CGA_IDCL,Number=1,Type=Float,Description="MEI InsertionLeftDnbCount: count of paired ends supporting insertion on 5' end of insertion point">
+##FORMAT=<ID=CGA_IDCR,Number=1,Type=Float,Description="MEI InsertionRightDnbCount: count of paired ends supporting insertion on 3' end of insertion point">
+##FORMAT=<ID=CGA_RDC,Number=1,Type=Integer,Description="MEI ReferenceDnbCount: count of paired ends supporting reference allele">
+##FORMAT=<ID=CGA_NBET,Number=1,Type=String,Description="MEI NextBestElementType: (sub)type of second-most-likely inserted mobile element">
+##FORMAT=<ID=CGA_ETS,Number=1,Type=Float,Description="MEI ElementTypeScore: confidence that insertion is of type indicated by CGA_ET/ElementType">
+##FORMAT=<ID=CGA_KES,Number=1,Type=Float,Description="MEI KnownEventSensitivityForInsertionScore: fraction of known MEI insertion polymorphisms called for this sample with CGA_IS at least as high as for the current call">
+##FORMAT=<ID=PS,Number=1,Type=Integer,Description="Phase Set">
+##FORMAT=<ID=SS,Number=1,Type=String,Description="Somatic Status: Germline, Somatic, LOH, or . (Unknown)">
+##FORMAT=<ID=GQ,Number=1,Type=Integer,Description="Genotype Quality">
+##FORMAT=<ID=HQ,Number=2,Type=Integer,Description="Haplotype Quality">
+##FORMAT=<ID=EHQ,Number=2,Type=Integer,Description="Haplotype Quality, Equal Allele Fraction Assumption">
+##FORMAT=<ID=CGA_CEHQ,Number=2,Type=Integer,Description="Calibrated Haplotype Quality, Equal Allele Fraction Assumption">
+##FORMAT=<ID=GL,Number=.,Type=Integer,Description="Genotype Likelihood">
+##FORMAT=<ID=CGA_CEGL,Number=.,Type=Integer,Description="Calibrated Genotype Likelihood, Equal Allele Fraction Asssumption">
+##FORMAT=<ID=DP,Number=1,Type=Integer,Description="Total Read Depth">
+##FORMAT=<ID=AD,Number=2,Type=Integer,Description="Allelic depths (number of reads in each observed allele)">
+##FORMAT=<ID=CGA_RDP,Number=1,Type=Integer,Description="Number of reads observed supporting the reference allele">
+#CHROM	POS	ID	REF	ALT	QUAL	FILTER	INFO	FORMAT	GS000016676-ASM
+1	1	.	N	<CGA_NOCALL>	.	.	END=10000;NS=1;AN=0	GT:PS	./.:.
+1	10001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=12000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.67:1.44:.:0:.:0:0.999:152
+1	10001	.	T	<CGA_NOCALL>	.	.	END=11038;NS=1;AN=0	GT:PS	./.:.
+1	11048	.	CGCACGGCGCCGGGCTGGGGCGGGGGGAGGGTGGCGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	11270	.	AGAGTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	11302	.	GGGCACTGCAGGGCCCTCTTGCTTACTGTATAGTGGTGGCACGCCG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	11388	.	AGGTGTAGTGGCAGCACGCCCACCTGCTGGCAGCTGGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	11475	.	ACACCCGGAGCATATGCTGTTTGGTCTCAGTAGACTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	11528	.	TGGGTTTAAAAGTAAAAAATAAATATGTTTAATTTGTGAAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	11650	.	TGGATTTTTGCCAGTCTAACAGGTGAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	11707	.	TGGGGCCTGGCCATGTGTATTTTTTTAAATTTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	11769	.	TGAGAATGACTGCGCAAATTTGCCGGATTTCCTTTGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	11841	.	CCGGGTATCATTCACCATTTTTCTTTTCGTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	11891	.	CTTTGACCTCTTCTTTCTGTTCATGTGTATTTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	11958	.	ACCGGGCCTTTGAGAGGTCACAGGGTCTTGATGCTGTGGTCTTCATC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12001	.	C	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=14000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:2.26:1.68:.:0:.:0:0.999:152
+1	12027	.	ACTGCTGGCCTGTGCCAGGGTGCAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12099	.	AGTGGGATGGGCCATTGTTCATCTTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12135	.	TGTCTGCATGTAACTTAATACCACAACCAGGCATAGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12187	.	AAGATGAGTGAGAGCATC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12238	.	CTTGTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12264	.	ACGTGGCCGGCCCTCGCTCCAGCAGCTGGACCCCTACCTGCCGTCTGCTGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12329	.	GCCGGGCTGTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12380	.	TCTGGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12551	.	GGTAGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12656	.	CCAGAGCTGCAGAAGACGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12734	.	TAGACAGTGAGTGGGAGTGGCGTCGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12780	.	GGCGTCTCCTGTCTCCTGGAGAGGCTTCGATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12829	.	GATCTTCCCTGTGATGTCATCTGGAGCCCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12936	.	CAGCAAACAGTCTGCAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	12976	.	TCAGAGCCCAGGCCAGGGGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	13113	.	AAGTGAGGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	13175	.	GGGGAGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	13270	.	CCAGTGATACACCCG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	13299	.	ACACGCTGTTGGCCTGGATCTGAGCCCTGGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	13358	.	ATTGCTGCTGTGTGGAAGTTCACTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	13410	.	ACCACCCCGAGATCAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	13610	.	GTGTTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	13653	.	AAACAGGGGAATCCCGAAGAAATGGTGGGTCCTGGCCATCCGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	13747	.	CTGCGTGGCCGAGGGCCAGGCTTCTCACTGGGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	13810	.	ACCTTCTTAGAAGCGAGACGGAGCAGACCCAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	13865	.	ACTAAAGTTAGCTGCCCTGGACTATTCACCCCCTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	13954	.	ACCTCCCCCACCTTCTTCCTGAGTCATTCCTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=16000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:2.19:1.78:.:0:.:0:0.999:152
+1	14108	.	ATCTTCTACCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14162	.	ACTATTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14207	.	GGAGACT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14245	.	GACTGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14351	.	AGACGTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14433	.	GCCGTTTTCTCTGGAAGCCTCTTAAGAACACAGTGGCGCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14495	.	ATGGAGCACAGGCAGACAGAAGTCCCCGCCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14539	.	TCAAGCCAGCCTTCCGCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14650	.	CAACGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14668	.	TCTGGGGGGGAAGGTGTCATGGAGCCCCCTACGATTCCCAGTCGTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14733	.	GGCTGCTGCGGTGGCGGCAGAGGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14809	.	CAGGTCCTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14904	.	GGAAGAAAAAGGCAGGACAGAATTACAAGGTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	14973	.	TGCGCCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15012	.	AGTGCCCACCTTGGCTCGTGGCTCTCACTGCAACGGGAAAGCCAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15115	.	GACACTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15186	.	CACCGGGCACTGATGAGACAGCGGCTGTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15237	.	CTCGGGGCCAGGGCCAGGGTGTGCAGCACCACTGTACAATGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15387	.	GGCGCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15422	.	ACAGCAGGCATCATCAGTAGCCTCCAGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15510	.	GACCGCTCTTGGCAGTCGAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15588	.	TCCCAAACCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15814	.	GCTGCTGCTTCTCCAGCTTTCGCTCCTTCATGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15862	.	TGCCGAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15893	.	TAGCAGAGTGGCCAGCCACCGGAGGGGTCAACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15953	.	GCCGGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	15988	.	TGCTCAGGCAGGGCTGGGGAAGCTTACTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16001	.	C	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=18000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:2.06:1.76:.:0:.:0:0.999:152
+1	16055	.	AAACGAGGAGCCCTGCG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16100	.	GTGTGGGGGCCTGGGCACTGACTTCTGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16203	.	CCCTGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16254	.	AGGGGTTTTGTGCCACTTCTGGATGCTAGGGTTACACTGGGAGACACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16374	.	GGAATGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16484	.	ATATTTGAAATGGAAACTATTCAAAAAATTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16531	.	TAACAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16567	.	GCACGCCAGAAATCAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16678	.	GGGAGTGGGGGTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16715	.	GGGGTGGTGGTGGGGGCGGTGGGGGTGGTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16798	.	AAGGTGTGTGACCAGGGAGGTCCCCGGCCCAGCTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16853	.	CCTACCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	16954	.	CATGAGGTCGTTGGCAATGCCGGGCAGGTCAGGCAGGTAGGATGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	17219	.	CCCACCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	17362	.	CTTCTACCTACAGAGGCGACATGGGGGTCAGGCAAGCTGACACCCGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	17481	.	GCCGAGCCACCCGTCACCCCCTGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	17594	.	C	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	17713	.	CACGCACACAGGAAAGTCCTTCAGCTTCTCCTGAGAGGGCCAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	17805	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	17882	.	GTGTGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	17942	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	17995	.	CGCCCGTGAAGATGGAGCCATATTCCTGCAGGCGCCCTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	18001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=20000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.79:1.98:.:0:.:0:0.999:152
+1	18083	.	TGAGGGGGCCCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	18140	.	TGGAAGCCTGGGCGAGAAGAAAGCTCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	18183	.	CAGGGCAGAGACTGGGCAGCCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	18253	.	GGGTATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	18350	.	CTCCGGCTCTGCTCTACCTGCTGGGAGAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	18496	.	CCTACTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	18562	.	GTCGAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	18640	.	GAAGGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	18691	.	GGCTACTGATGGGGCAAGCACTTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	18742	.	ACAATGTGGCCTCTGCAGAGGGGGAACGGAGACCGGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	18831	.	GAACTGCCCCTGCACATACTGAACGGCTCACTGAGCAAACCCCGAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	18930	.	GGTGCGGGGTGGGCCCAGTGATATCAGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	19001	.	TGCATCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	19048	.	CTGATGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	19169	.	CACAACATCCTCCTCCCAGTCGCCCCTGTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	19265	.	GCACTCACCGGGCACGAGCGAGCCTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	19303	.	GGATGAGAAGGCAGAGGCGCGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	19387	.	AGGCGCGACTGGGGTTCATGAGGAAAGGGAGGGGGAGGATGTGGGATGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	19589	.	CCGTGCCCTAAAGGGTCTGCCCTGATTACTCCTGGCTCCTTGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	19680	.	AAGCGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	19772	.	CGGGACCACCACCCAGCGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	19855	.	TGTCTTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	19939	.	TTCGAGGTCCACAGGGGCAGTGGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20001	.	C	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=22000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:2.01:1.69:.:0:.:0:0.999:152
+1	20096	.	AACAGAGAGTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20133	.	GAGTCCCAGGGGCCAGCACTGCTCGAAATGTACAGCATTTCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20188	.	TTATTAGCCTGCTGTGCCCGGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20242	.	CAGGATTTTGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20301	.	GGAGAGAACATATAGGAAAAATCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20385	.	CACGGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20463	.	TAAGCTGGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20508	.	GCCAGAGGGTAGACTGCAATCACCAAGATGAAATTTACAAGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20592	.	TATAAATACAGAAGGTGGAGGGAACTTGCTTTAGACACAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20708	.	AAAGGCAATGAGATCTTAGGGCACACAGCTCCCCGCCCCTCTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20810	.	AAAACAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20859	.	CAGAGGGTGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20911	.	AGCAGGAGGAGAGAGCACAGCCTGCAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20957	.	CACCTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	20982	.	ACGCCAGTGAGGCCAGAGGCCGGGCTGTGCTGGGGCCTGAGCCGGGTGGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21073	.	GAGGAGCATGTTTAAGGGGACGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21118	.	ACCGAAAAAGCCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21169	.	AGGAGGGGCAAGTGGAGGAGGAGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21221	.	GTCGTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21316	.	CTTGCAAGTCCCCTGTCTGTAGCCTCACCCCTGTCGTATC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21408	.	CTTGTCCCTTCCGTGACGGATGCCTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21500	.	CACGCCTGAATCAACTTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21580	.	C	T	.	.	NS=1;AN=2;AC=2;CGA_XR=dbsnp.83|rs526642;CGA_FI=653635|NR_024540.1|WASH5P|INTRON|UNKNOWN-INC;CGA_SDO=5	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/1:.:PASS:126:126,168:126,168:48,48:-168,-126,0:-48,-48,0:1:1,1:0
+1	21623	.	ACACAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21707	.	CCCGAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21766	.	CCCCTCCCACCCCTGTGCAGGCCGGCCTTCGCGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21837	.	CCTCCCTCCAAGCCTGCAGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21870	.	CCCTGCCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	21988	.	GCAATGGCCCCATTTCCCTTGGGGAATCCATCTCTCTCGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=24000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.65:1.60:.:0:.:0:0.999:152
+1	22067	.	GCTCCTCAGTCTAAGCCAAGTGGTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22109	.	CCCATTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22152	.	GGGATGAGTGAGTGTGGCTTCTGGAGGAAGTGGGGACACAGGACAGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22246	.	CGAGAGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22371	.	TTAATTTTTGCTTAGCTTGGTCTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22408	.	GGCGTGCCACCAATTCTTACCGATTTCTCTCCACTCTAGACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22491	.	TCTCGCCCTATGTGTTCCCATTCCAGCCTCTAGGACACAGTGGCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22651	.	TGAGAGGCATCTGGCCCTCCCTGCGCTGTGCCAGCAGCTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22755	.	CATCCCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22822	.	AGACGCCAAAAATCCAGCGCTGCCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22861	.	CCACGCAGTCCCCATCTTGGCAAGGAAACACAATTTCCGAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22945	.	CCATAATC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	22977	.	TGCATCCTCTTCCCTAGGTGTCCCTCGGGCACATTTAGCACAAAGATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23044	.	GCACTTTGTTACTATTGGTGGCAGGTTTATGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23094	.	GTACGGGTCAAGATTATCAACAGGGAAGAGATAGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23171	.	TTTGCATGTTTTGATTAATTTAATATTTAAAATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23252	.	CACCGAGGCTTAGAGGGGTTGGGTTGCCCAAGGTTACAGAGGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23356	.	TCACTGTGTGTCCCCTGGTTACTGGGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23391	.	ACAAACTCGGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23503	.	CTGGCGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23528	.	GGCAGGGATGGCTTGGACCACGAGAGGCACCTGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23581	.	CCCACTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23643	.	TCAGTTTGCTTATGGCCAAAGACAGGACCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23697	.	TTTACCAAAAAAAGAGCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23786	.	AGCACTGCCAATACAAGAAGCTGCAGCCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23827	.	CCCTCAATGGCCACTCCGTGCTCCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23878	.	CCACCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	23975	.	G	A	.	.	NS=1;AN=1;AC=1;CGA_XR=dbsnp.100|rs2748067&dbsnp.131|rs76046194;CGA_FI=653635|NR_024540.1|WASH5P|INTRON|UNKNOWN-INC;CGA_RPT=L2b|L2|53.1;CGA_SDO=5	GT:PS:FT:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/.:.:PASS:102,.:102,.:40,.:-102,0,0:-40,0,0:0:0,.:0
+1	24001	.	G	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=26000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.46:1.70:.:0:.:0:0.999:152
+1	24031	.	CCCCATC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24095	.	GAATCCTGGCTCTGTCACTAGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24127	.	CAGCCCTTCTGTGCCTCAGTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24194	.	TGAGTTAATGCACTCAAATCAATGGTTGTGCACGGTTTATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24266	.	AGACCTTGTCACAACTGTTATTGAAGAACTAATCATCTATTGCTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24344	.	TCCAGGTGGAGAGGTATGTTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24416	.	CACTGCTGGGTAAATATTTGTTGGCTGCAGGAAAACGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24480	.	AAAAGCATGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24508	.	CCACAGGAAACCAGGAGGCTAAGTGGGGTGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24635	.	GACCGGGATTCCCCAAGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24701	.	GCCCTCTCATCAGGTGGGGGTGAGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24762	.	TTCTGCAGGTACTGCAGGGCATCCGCCATCTGCTGGACGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24829	.	TGAAGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24871	.	TCCTCACAGGAGTCATGGTGCCTGTGGGTCGGAGCCGGAGCGTCAGAGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	24936	.	CACGCCCCCACCACAGGGCAGCGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25041	.	TTGCTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25062	.	ATAGATGGGACTCTGCTGATGCCTGCTGAGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25108	.	CAGGGCCCGGGACTGGGGAATCTGTAGGGTCAATGGAGGAGTTCAGAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25215	.	TACCTTGTCTCAGTTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25305	.	GGGGTAGCAGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25344	.	TACACAGTTCTGGAAAAGCACAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25437	.	GGGCAGTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25496	.	CTTAGGGGGACAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25530	.	GGTGGAGGACAGGAAGGAAAAACACTCCTGGAATTGCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25609	.	CTCTCCCTGGTGCCACTAAAGCAGCAATCACACTGCAGACAGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25730	.	CTCACTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25771	.	ACTGTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25871	.	TTTCACTTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25931	.	TTCGATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	25970	.	GAGACGTGGTTATTTCCAATAATAATTTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=28000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.37:1.49:.:0:.:0:0.999:152
+1	26010	.	TAACGCACCACACCAACATCTTCACCCAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26057	.	CTCCCGCTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26263	.	ACCCAACCCTCTGAGACCAGCACACCCCTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26339	.	GTTTGCTGGCTGTCCTAACTTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26443	.	ATTTCTTGTTAGTGTGTGTGTGTTTGCTCACACATATGCGTGAAAGAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26501	.	ACAGATCTCCTCAAAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26561	.	TGAGTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26595	.	GATCATCTGTTAGGTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26648	.	ACTAGCCAGGGAGAGTCTCAAAAACAAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26697	.	CTACTCCAGTCATGGGTACAAAGCTAAGGAGTGACAAATCCCTCTTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26792	.	GCCGGGCGCAGCGGCTCACGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26835	.	GGCGAAGGCAGGCAGATCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26887	.	ACATGGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26926	.	GCCAGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26954	.	CCCCGCTACTCGGGAGGCTGAGGAAGGAGAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	26992	.	AACCAGGAAGGTGGAGGTTGCAGTGTGCCAAGATCGCGCCATGGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27044	.	CCTAGGCAACGAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27103	.	AAAGAAACAGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27136	.	AACCGCAAGCGGTCTTGAGTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27172	.	TCCTTGGGGAAGTACTAGAAGAAAGAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27227	.	CACTCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27274	.	CCTTAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27313	.	CATGCAGCCACTGAGCACTTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27361	.	GCCATAAGTGTAAAATATGCACCAAATTTCAAAGGCTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27428	.	TTTATATTGATTACGTGCTAAAATAACCATATTTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27486	.	TATCACTAATTTCATCTGTTTCTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27537	.	TTAAATATTTCTTTTCTTTTTCTTTCCTCTCACTCAGCGTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27604	.	GCTGTTTTTGGGCAGCAGATATCCTAGAATGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27664	.	TCATAAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27720	.	TGACCATGTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27777	.	ACAGTATGACTGCTAATAATACCTACACATGTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27840	.	TTAACTCTTATTATCAGTGAATTTATCATCATCCCCTATTTTACATAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27904	.	AGACCAAATAACATTTTTTCAACATCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	27965	.	CTGTCGTCTGAATTCCAAGCTTTTTGTTATTTATTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=30000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.40:1.68:.:0:.:0:0.999:152
+1	28039	.	GCCCAAACATTTTGTTAGTAGTACCAACTGTAAGTCACCTTATCTTCATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28110	.	AATTAGATCTGTTTTTGATACTGAGGGAAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28174	.	TGTGGTCAACACTTTCGTTACTTTAGTATACATCACCCCAATTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28244	.	TAGGTAGTAGTATCTATTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28324	.	TAGTTGCTCATCTGAAGAAGTGACGGACCACCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28364	.	AGTGGACAGACAGTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28391	.	GACAGGGGATTTTGTTGGCGGAAAAAAAAATTTATCAAAAGTCGTCTTCTATCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28474	.	AGTTCCACAGTGGGTAACTGTAATTCATTCTAGGTCTGCGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28555	.	CCACAAATACCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28582	.	ATGGTGGTTTTTTTTTTTTTTTGCAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28660	.	CGCTCAATATTTCTAGTCGACAGCACTGCTTTCGAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28748	.	ACCGCGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28811	.	TCCAGGGTCTCTCCCGGAGTTACAAGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28860	.	CAACGCGGTGTCAGAGAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28906	.	TCCGGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	28950	.	GAACCCGGCAGGGGCGGGAAGACGCAGGAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29011	.	CGGGTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29061	.	GCCGGGTGCAGGGCGCGGCTCCAGGGAGGAAGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29121	.	GGCGGTCGGGGCCCAGCGGCG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29153	.	GGAGCCGGGCACCGGGCAGCGGCCGCGGAACACCAGCTTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29253	.	CGGGTCCCCTACTTCGCCCCGCCAGGCCCCCACGACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29359	.	CGCTCTGCCGGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29390	.	GCCGCCCCCAGTCCGCCCGCGCCTCCGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29430	.	CCGCTCGCCCTCCACTGCGCCCTCCCCGAGCGCGGCTCCAGGACCCCG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29495	.	CCTGTCGGGCCG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29568	.	CATGCGTTGTCTTCCGAGCGTCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29630	.	TCCTAGACCTCCGTCCTTTGTCCCATCGCTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29693	.	CCAACCTCGGCTCCTCCGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29721	.	GCCCGGGGTGCGCCCCGGGGCAGGACCCCCAGCCCACGCCCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29787	.	TACGCCTTGACCCGCTTTCCTGCGTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29846	.	GGGCAGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29874	.	CCCACCCCCCTTTAAGAATTCAATAGAGAAGCCAGACGCAAAACTACAGATATCGTATGAGTCCAGTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	29958	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30001	.	G	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=32000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.52:1.49:.:0:.:0:0.999:152
+1	30024	.	AGCTCGTGTTCAATGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30076	.	AAATGAGTGGTAGTGATGGCGGCACAACAGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30237	.	TTTTAAAAAGTTAAATATAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30285	.	GCAGTTGTCCCTCCTGGAATCCGTTGGCTTGCCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30359	.	AAAGACAGGATGCCCAGCTAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30408	.	TTCGTAGCATAAATATGTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30485	.	TTCAGAATTAAGCATTTTATATTTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30525	.	CCACCCTACTCTCTTCCTAACACTCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30567	.	TGTCCGCCTTCCCTGCCTCCTCTTCTGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30633	.	CTCGCTGGCTGCAGCGTGTGGTCCCCTTACCAGAGGTAAAGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30725	.	AATGCTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30776	.	CCTTTGGTAGGTAATTACGGTTAGATGAGGTCATGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30852	.	TTGTCTCTGTGTCTCCCTCTCTCTCTCTCTCTCTCTCTCTCATTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30907	.	CATTTCTCTCTCTCTCGCTATCTCATTTTTCTCTCTCTCTCTTTCTCTCCTCTGTCTTTTCCCACCAAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	30982	.	TGCGAAGAGAAGGTGGCTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31026	.	ACCGGGAACCCGTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31125	.	TTTTGTTTTGACTAATACAACCTGAAAACATTTTCCCCTCACTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31248	.	GCCTGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31282	.	CACAGGCTCAGGGATCTGCTATTCATTCTTTGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31381	.	GCCCTGCCTCCTTTTGAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31431	.	AATCTGGCTGGGCGTGGTGGCTCATGCCTGTAATCCTAGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31484	.	GACGCGAGAGGACTGCTTGAGCCCAAGAGTTTGAGAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31552	.	TACAAAAATAAAATAAAATAGCCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31647	.	GATCGAGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31669	.	GATTGTACCACTGCACTCCAGGCTGGGCGACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31716	.	TCAGAAAAAAAAAAAAAAGTAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31790	.	CACGATGCCTGTGAATATACACACACACCACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31856	.	TGCACTGCTAGGCACCACCCCCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	31938	.	GTTCCCTACCTAATCTACTGACAGGCTCATCCCCGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32000	.	TGCAGTGGGAATCCTGGACCTCAGCCTGGACAAAGAACAGCTGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32001	.	G	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=34000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.55:1.39:.:0:.:0:0.999:152
+1	32064	.	CACAGAAGCTCTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32093	.	AGCTGGGCTGAGCGGGCCTGGGAATTAAGGCTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32147	.	TTGCTGAAGCTTGCCACATCCCCCAGCCTCCTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32227	.	GAGTGAAGAAAATGTGACAGGGTGTCCTAAGCCCCGATCTACAGGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32322	.	GCCTCTAGCTTTTGTGCTACAGTTCTGGGAACAGACTCCTCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32374	.	CCACTTCCCTCCGCAGCATTAGATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32455	.	ATGGAAATGTCCTGCTCTCTAAACAGATAGACAGAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32584	.	TGCACGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32646	.	CGGTGACTGTGTTCAGAGTGAGTTCACACCAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32733	.	CAGCCCAGGAACCTCCCCTTATCGGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32815	.	TGCCATGTGGGTTGTTCTCTGAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32891	.	CCTCGGCTGGAGTCCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	32971	.	TATGTAATAACTGAATCTGTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33051	.	CCATAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33108	.	TCATAAAAAGGAAGGCAGAGCTGATCCATGGCACCATGTGACAGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33176	.	GGAAGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33258	.	AGCCGAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33329	.	GAGCTGATGAAAATGTTTTGGAACTACATAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33377	.	CATGGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33398	.	CACTGATTGTTCAATTTAAAATGGTCAAACTTATATGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33443	.	CTCCATTAAAAAAAAAAAAAAAGGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33492	.	AATCCCAACACTTTGGAAAAAGGTGAAAGTTTTTTTTTCTTTTTTTTTTTATAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33552	.	GTTCTAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33653	.	TCTTCTAATGCTATCCCTCCCCCAGCCCCCCACCCAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33703	.	TGTATGATGTTCTCTGCCCCATGTCCAAGCGTTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33744	.	TCAATTCCCACCTGTGAGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33797	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33868	.	ATGGCTGCATAGTATCCCATGGTATATATGTGCCACATTCTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33925	.	ATTGATGGACATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	33958	.	GCTATTGTGAATACTGCCACAATAAACATACAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34001	.	C	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=36000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.23:1.45:.:0:.:0:0.999:152
+1	34026	.	CTTTGGGTATATACCCTAAGACCTGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34125	.	TGGGTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34291	.	TCTCACATCTTCTTGGCCAGCACTGGACCAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34358	.	TATGAGAAAGAAGGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34404	.	CATCTGCTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34452	.	AGAAGGCTTTCTGGACGGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34513	.	TGGTGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34581	.	AGGCAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34631	.	AAGTTTATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34704	.	TGGTTCTGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34740	.	GCCGTGCTCCTTGGAGGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34767	.	AGGCGGAGGACACGTGGGAGGTTTTAGGGACAAGCCTGGAGGCAGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	34968	.	GTCAGGCAGGGAGTGGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	35030	.	GGTGGAGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	35076	.	GACCACGTGCTGGATGTCACGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	35116	.	GCCGGGTTAGGCACCTGGTGTTTTACGTACATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	35160	.	GTGAGGGCATCCGACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	35229	.	TTAGAGCTTAATCGTGTTCAAAATACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	35364	.	GTCGGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	35407	.	GGGAGGCTGAGGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	35482	.	CTATAAAAAATAAATAAATAAATAAAAACAAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	35656	.	GGCTTAGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	35755	.	TCACGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=38000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.06:1.37:.:0:.:0:0.999:152
+1	36070	.	CCCCGTTGTGTGGGAGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36125	.	AGGGAGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36158	.	TGCAGCTGGCTCATTCCCATATAGGGGGAGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36229	.	GGGGAGGCCAGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36258	.	GGGTGGCTCTGAGGGGGCTCTCAGGGGTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36344	.	AAGTTTGGAAAAAAAAAAAAACCCAGCCTGGCGGAAAGAATTTAAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36483	.	GTCATCCTTCCCCAACACATCCCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36521	.	CAAGCCTCTCCCACCCAAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36590	.	AGACATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36722	.	TGATTCTGTGGTATGTTAATGTTTATGCATAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36786	.	GAGCCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36851	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36888	.	GGCTAGTTGTTTGAATGTCTGCATGAAAAAGCGGACGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36960	.	GACCGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	36996	.	CTGCACTATTAATTTGTTTTTTAGCTAAGGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37052	.	CCACCCAGTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37103	.	TGAACCTACCTTTTCAATGTAAATTCAGTGAAATCTAAGTGCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37236	.	AGGGATTTTTTTTTTGTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37296	.	AATGATAATCTTTTGGATATATTGGGTTAAATAAATTTATTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37362	.	GTTTAATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37506	.	AGAGGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37560	.	ACAGAGATAACTCCAACCCTTAAGAAGGTGTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37624	.	TACTGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37688	.	GCCGTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37714	.	GTGGTCTCACCTCCGGCAGTATCACCACCACTGGGCACAAGCTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37769	.	CAACTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37791	.	GTACTCCCAGTGTTCACACCATGCTGCACTCACAGAAGACTCTTCGTTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37882	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37941	.	GGGGAGGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	37962	.	AGCCAGGAGTCTCATCCCCTGGGGAAGTTCCAGGGACCCCTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	38001	.	C	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=40000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.37:1.47:.:0:.:0:0.999:152
+1	38049	.	CAGAGCCTGCCTTCCACGTGGGTTTGACAGGAGCCTCCTAACTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	38140	.	GCCAGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	38221	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	38232	.	A	G	.	.	NS=1;AN=2;AC=2;CGA_XR=dbsnp.131|rs77823476&dbsnp.86|rs806727;CGA_FI=645520|NR_026818.1|FAM138A|TSS-UPSTREAM|UNKNOWN-INC;CGA_SDO=6	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/1:.:PASS:87:195,87:195,87:48,48:-195,-87,0:-48,-48,0:1:1,1:0
+1	38339	.	GGACAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	38378	.	AAAGTGGTCTCCTGCAGTTACGTGGCAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	38481	.	TTCTTCTTACTGCTTATAATATAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	38592	.	GTCTCCCCACATGGAAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	38736	.	AGATTACAAGGGTGTACCATGCAGAACCTCTCCACCAAACCTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	38821	.	TTGGCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	38907	.	C	T	.	.	NS=1;AN=2;AC=2;CGA_XR=dbsnp.108|rs3874156&dbsnp.131|rs75829199&dbsnp.86|rs806726;CGA_FI=645520|NR_026818.1|FAM138A|TSS-UPSTREAM|UNKNOWN-INC;CGA_RPT=MLT1E1A-int|ERVL-MaLR|38.5;CGA_SDO=6	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/1:.:PASS:94:172,94:172,94:48,48:-172,-94,0:-48,-48,0:13:9,9:4
+1	38998	.	AAAGAAGACTGTCAGAGACCCCAAACTCTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	39140	.	AATCTTCCCACATCTTAAAACCTGTTTAGGGAACACCAGCATCTGTCCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	39217	.	TCCTTCCCCTGCTGCCTCTTTCTGAACAGCAATGTCTCAAGCTTTACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	39289	.	GGGGCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	39403	.	CAAGAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	39484	.	GTGACCCCCACAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	39575	.	AGGCGGTATATATGTGATTCATGTACTGATCAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	39625	.	GCTGGATGCAGTGGCTCGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	39654	.	CCAACACTTTGGGAGGCTGAGGCGGGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	39702	.	TCGAGACCAGGCTGGCCAACATGGCAAAACCCCGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	39827	.	ACTCAGGAGGTGGAGGTGGCAGTGAGCCAAGATCGTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	40001	.	G	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=42000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.33:1.54:.:0:.:0:0.999:152
+1	40026	.	GGAGCTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	40090	.	CATAGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	40156	.	GAACCCAGTGCTGGCTGACACCCTGATGGCACCTTACAGAGGACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	40234	.	CTGGGGAACACTGGGTCGTATTTGCAGCTGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	40295	.	TGGGAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	40386	.	CCGCAGCCACGCTGGCCCTCTGCTGTTCTTCGAAGCCACCAGGGCTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	40491	.	GTATATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	40604	.	ACATGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	40636	.	TTCCTTTTTTTTTTTTTTTTTTTGACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	40703	.	CTCGGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	40890	.	ACTGGTCTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	41025	.	TCTTGGGAATATTAAGTGGAGAGGGGTACGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	41215	.	TTCTTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	41253	.	CCTCATGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	41376	.	AAAGGTATAGCAATATTTCTATTTCCTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	41475	.	GGAAGCCAAAATGACAGGGAGCTACTAAAACTTTATTCTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	41668	.	AGGACCCAATATCTTACAATGTCCATTGGTTCAAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	41757	.	GGCATTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	41839	.	AGTATAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	41981	.	A	G	.	.	NS=1;AN=2;AC=2;CGA_XR=dbsnp.86|rs806721;CGA_RPT=ERVL-E-int|ERVL|47.4;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/1:.:PASS:109:109,238:109,238:48,48:-238,-109,0:-48,-48,0:6:2,2:4
+1	42001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=44000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.57:1.37:.:0:.:0:0.999:152
+1	42036	.	GAACAAATTGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	42198	.	CATGCTGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	42262	.	ATTGTGCAAGCATAAGTGGCTGAGTCAGGTTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	42343	.	ACATAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	42522	.	TTATACT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	42574	.	TTACGCTTTTCTTAAACACACAAAATACAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	42662	.	AATCAATTAGCAATCAGGAAGGAGTTGTGGTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	42736	.	AAATTATTCACAATAAAAAAAAAGATTAGAATAGTTTTTTTAAAAAAAAAGCCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	42821	.	AGGTATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	43039	.	TTAGGCAAGGAAGCAAAAGCAGAAACCATGAAAAAAGTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	43201	.	ATATCAATAACCACAACATTCAAGCTGTCAGTTTGAATAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	43370	.	GAAACCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	43583	.	ACACTGGTAAAAAAAATGAAAGCTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	43662	.	ATATCGTACTAAAAGTCCTAGCCAGGACAATTAGACAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	43793	.	CAGCAAAAAAAAAAAAAAAACTAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	44001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=46000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.42:1.36:.:0:.:0:0.999:152
+1	44162	.	TTTGACAGAAATAAAAAAAAAATTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	44329	.	AAACACACAGATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	44669	.	AAATAGACAAATGAGACTATGCCAAATTAAAAAATTTCTAACAACAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	44752	.	GAATGGGAGAAATATTTGCAAACTACTCATCCAACCGGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	44832	.	AGTAAAATAAATAAATAAATAAATAAATAAATAAATTAAATAAATTATTTAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	45028	.	TGGCTATTATCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	45084	.	GAACCCTTGCATCATGTACAAATTAAAAATAGAAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	45244	.	ACCTAAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	45517	.	AAAGGAAAAAAATTCAATTAGTAGGATTACATTCAGGGGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	45623	.	TATTGTAAATGTTAATATGAGGTAATATATGTGTTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	45753	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	45758	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	45791	.	GATTAAAAAAATTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	45853	.	TTTTAAATATAATTTAAACCAAATTTAAAATAAGCATATAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	45974	.	TAAATTTTAAAATATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	46001	.	G	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=48000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.56:1.63:.:0:.:0:0.999:152
+1	46051	.	CACTCAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	46085	.	TATGTCAGATCATGAATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	46399	.	TAACTTTTTTTTTTTTTTGAGCAGCAGCAAGATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	46670	.	A	G	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2548905;CGA_RPT=MER45A|hAT-Tip100|29.0;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:VQLOW:24:24,24:24,24:5,24:-24,0,-24:-5,0,-24:22:0,22:22
+1	46868	.	GCCAAAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	46922	.	AAAGTAAATATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	47108	.	G	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2531241;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:PASS:75:75,75:75,75:32,36:-75,0,-75:-32,0,-36:16:1,15:15
+1	47489	.	TCCCTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	47658	.	AAATCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	47693	.	GCATCAATGGGTCACTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	47932	.	TCTCCTCCACTTTTCTGTTTTCCTCCTATCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	47995	.	ACAATCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	48001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=50000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.72:1.41:.:0:.:0:0.999:152
+1	48445	.	GAACGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	48934	.	CAGTATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	49240	.	CAAGCTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	49269	.	ACCGCATGTTCTCACTTATGAGCGTGAGATAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	49431	.	CTGTACAACGAACCCCCAGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	49479	.	CACGTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	49506	.	AGTCAAAAAGAAAAAGAAAAAAAGAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	49829	.	AAAGCAAAACAAACAAACAAACAAAACAAAACACTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	50001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=52000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.47:1.56:.:0:.:0:0.999:152
+1	50169	.	GGTATGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	50482	.	GTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	50888	.	TGGTATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	51456	.	AGCGGAAGAGTAAGTCTTTGTATTTTATGCTACTGTACCTCTGGGATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	51617	.	AGCACTTTGGGAGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	51665	.	GACCATCCTGGCTAACACGGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	51742	.	TGCGGTCCCAGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	51803	.	T	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.129|rs62637812;CGA_RPT=AluY|Alu|7.7;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:PASS:82:112,82:111,81:38,38:-112,0,-82:-38,0,-38:20:9,11:11
+1	51861	.	CCTCAAAAAAAAAAAAAGAAGATTGATCAGAGAGTACCTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	52001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=54000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.20:1.19:.:0:.:0:0.999:152
+1	52058	.	G	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.129|rs62637813&dbsnp.131|rs76894830;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:VQLOW:24:24,26:24,26:3,25:-24,0,-26:-3,0,-25:9:4,5:5
+1	52134	.	CTTGTCTAATTGTTATTAATAATTAATAAATAACTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	52182	.	TTATTAATAATAACTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	52206	.	ATGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	52228	.	AATAACTT	AC	.	.	NS=1;AN=2;AC=1;CGA_RPT=AT_rich|Low_complexity|3.1;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1|0:52228:PASS:43:121,43:118,40:24,20:-121,0,-43:-24,0,-20:27:11,16:16
+1	52238	.	T	G	.	.	NS=1;AN=2;AC=2;CGA_XR=dbsnp.100|rs2691277&dbsnp.134|rs150021059;CGA_RPT=AT_rich|Low_complexity|3.1;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1|1:52228:PASS:75:275,75:272,72:52,48:-275,-75,0:-52,-48,0:26:26,26:0
+1	52727	.	C	G	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2691278;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:PASS:78:78,78:78,78:27,37:-78,0,-78:-27,0,-37:22:1,21:21
+1	52797	.	CAGGTAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	53006	.	AGACACTTACAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	53184	.	TTTTTATGCCATGTATATTTCTGTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	53385	.	AGTGCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	53425	.	ACATGGCTACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	53705	.	AGTAAGCATATAGATGGAATAAATAAAATGTGAACTTAGGTAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	53786	.	TTAATTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	53955	.	GTCAAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	53998	.	CCTACAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	54001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=56000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.63:1.53:.:0:.:0:0.999:152
+1	54043	.	C	T	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2531228;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:VQLOW:35:35,35:35,35:10,27:-35,0,-35:-10,0,-27:26:5,21:21
+1	54108	.	AAGGATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	54175	.	AGCTTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	54287	.	ACACATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	54351	.	AACCGTACCTATGCTATTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	54586	.	T	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.131|rs79600414;CGA_RPT=L2|L2|49.7;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:VQLOW:39:39,39:33,33:3,17:-39,0,-39:-3,0,-17:64:15,49:49
+1	54705	.	CTTGTATTTTTCTTTCTTTCTTTCTTTCTTTCTTTCTTTCTTTCTTTCTTTCTTTCTTTCTTTCTTTCTTCCTCCTTTTCTTTCCTTTTCTTTCTTTCATTCTTTCTTTCTTTTTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	54841	.	GTTGCTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	55082	.	CAATAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	55164	.	C	A	.	.	NS=1;AN=1;AC=1;CGA_XR=dbsnp.103|rs3091274;CGA_RPT=L2|L2|49.7;CGA_SDO=2	GT:PS:FT:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/.:.:PASS:44,.:44,.:18,.:-44,0,0:-18,0,0:18:18,.:0
+1	55296	.	ATGCGACCTTCCCACTTAAAATCCTACTATTTACGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	55378	.	GCTGAAGACACTTCACTTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	55425	.	CATGGTATAGTAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	55484	.	AACATAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	55542	.	ACTCCAAAATCTATCAACTCTGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	55813	.	GTCGTGTTCACCTCTATCACATCATAAATATAGCAAACAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	55926	.	T	C	.	.	NS=1;AN=2;AC=2;CGA_XR=dbsnp.101|rs3020698&dbsnp.121|rs13343114;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/1:.:PASS:31:65,31:65,31:22,30:-65,-31,0:-30,-22,0:23:23,23:0
+1	55973	.	TTCTTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	56001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=58000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.42:1.69:.:0:.:0:0.999:152
+1	56154	.	GTTAGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	56295	.	ATACGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	56378	.	TTATGTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	56482	.	TTTCACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	56635	.	ACACTTCTTATTCTGCTGCTGTTCTAGAGAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	56799	.	T	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2691309;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:VQLOW:31:31,31:31,31:8,26:-31,0,-31:-8,0,-26:23:2,21:21
+1	56984	.	ATCCAAAAAAAATACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	57155	.	CCTGTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	57246	.	C	G	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2691313;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:VQLOW:26:26,26:26,26:3,17:-26,0,-26:-3,0,-17:34:5,29:29
+1	57289	.	TTCCGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	57373	.	TCCCTCCCCCTATTTCATCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	57792	.	TTTCTTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	57853	.	AACTAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	57892	.	TGTCCCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	57952	.	A	C	.	.	NS=1;AN=2;AC=2;CGA_XR=dbsnp.100|rs2691334&dbsnp.135|rs189727433;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/1:.:PASS:81:205,81:205,81:48,48:-205,-81,0:-48,-48,0:13:12,12:1
+1	57987	.	TGTCTGATCTCAGCTATTTCCATCCTATTTGAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	58001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=60000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.85:1.47:.:0:.:0:0.999:152
+1	58176	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	58211	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	58595	.	ACTATAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	58909	.	TAAATACT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	58986	.	TCTCTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	59051	.	A	G	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2691352;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:PASS:113:147,113:147,113:44,42:-147,0,-113:-44,0,-42:26:11,15:15
+1	59131	.	ATGCAAAAAGTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	59319	.	ATCCCACCATACCTCATTATCACACCTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	59498	.	T	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2854666&dbsnp.131|rs76479716;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:PASS:122:122,122:112,112:13,34:-122,0,-122:-13,0,-34:50:32,18:18
+1	59615	.	CCTTCCCCTGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	60001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=62000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.37:1.68:.:0:.:0:0.999:152
+1	60276	.	TTGTTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	60405	.	TTACGTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	60718	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	60726	.	C	A	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2531295&dbsnp.131|rs77618875&dbsnp.135|rs192328835;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1|0:60726:PASS:77:86,77:80,71:12,27:-86,0,-77:-12,0,-27:43:31,12:12
+1	60788	.	AATACATGCATATTGTGGAGATAGAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	61018	.	TTCGTTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	61216	.	TGATTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	61289	.	A	AG	.	.	NS=1;AN=2;AC=1;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:PASS:57:57,57:57,57:22,26:-57,0,-57:-22,0,-26:11:1,10:10
+1	61347	.	TTGTAAAAAAAAAAATATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	61442	.	A	G	.	.	NS=1;AN=2;AC=2;CGA_XR=dbsnp.100|rs2531261&dbsnp.129|rs62637818&dbsnp.131|rs74970982;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/1:.:PASS:31:31,62:0,9:14,0:-62,-31,0:-14,0,0:11:11,11:0
+1	61448	.	A	.	.	.	NS=1;AN=1	GT:PS:FT:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL	0/.:.:PASS:127,.:74,.:0,.:0:0
+1	61477	.	TTTGTTTACCATTATTACTCTTGGTATTTTTAAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	61576	.	TCCGTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	61675	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	61848	.	TAATAATTGTAAAACTTTTTTTTCTTTTTTTTTGAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	61984	.	CCCAAGTAGCTGGGACTACAGGCATGCACCACCATGCCCAGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	62001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=64000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.32:1.47:.:0:.:0:0.999:152
+1	62203	.	T	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.125|rs28402963;CGA_RPT=L1M5|L1|41.7;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:PASS:90:90,90:90,90:37,39:-90,0,-90:-37,0,-39:18:5,13:13
+1	62236	.	ACATACACACACACACACACATATCTGTATATACAAATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	62295	.	ATTCTTCATTTCATTTGTTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	62536	.	TTTTTAAAGATTCTGTATTTTTTAAACCATTTATTTGTATATGTTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	62678	.	CCAGTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	62774	.	TTAACCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	62877	.	TTTTCAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	63071	.	AAAACTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	63236	.	TTGCATGATACAAAAGTTCTTTATCCATGTTATGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	63356	.	GAATCCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	63413	.	TGCTATGTCTCAGTTTGTTTTTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	63466	.	ATGTGGGGAGCTTTTATTGTGATTTTCCTCGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	63513	.	TGCATGGACACTTATGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	63693	.	TTGTTTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	63733	.	TCCCTACTAAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	63792	.	G	T	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.101|rs2907079;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:PASS:57:57,57:57,57:20,33:-57,0,-57:-20,0,-33:23:8,15:15
+1	63908	.	ACAGGAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	64001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=66000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.68:1.49:.:0:.:0:0.999:152
+1	64122	.	CTACCTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	64168	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	64200	.	AGTATTTTTATGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	64277	.	ATGAGTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	64476	.	GAAGGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	64510	.	GGTGGGGGGAAGGGGGAGGGATAGCATTAGGAGATATAACTAGTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	64580	.	CACACCCACATGGCACATGTATACATATGTAACTAACCTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	64761	.	TCTCGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	64973	.	ATTCCTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	65250	.	AAAAAGCACCTTTAGACTCAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	65526	.	TGCCTCATTCTGTGAAAATTGCTGTAGTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	65588	.	AGACAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	65742	.	GAGAAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	65794	.	TGCTGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	66001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=68000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:2.27:1.67:.:0:.:0:0.999:152
+1	66094	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	66157	.	C	<CGA_NOCALL>	.	.	END=66634;NS=1;AN=0	GT:PS	./.:.
+1	66734	.	AAGGACT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	66790	.	TCTAATTTTTTTTGAATAATTTTTAAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	67069	.	TTAATTTTATGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	67223	.	C	.	.	.	NS=1;AN=1	GT:PS:FT:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL	0|.:67223:PASS:42,.:39,.:0,.:0:0
+1	67242	.	A	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2531263&dbsnp.131|rs77818189;CGA_FI=79501|NM_001005484.1|OR4F5|TSS-UPSTREAM|UNKNOWN-INC;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	0|1:67223:PASS:42:42,42:39,39:20,4:-42,0,-42:-4,0,-20:59:34,25:34
+1	67445	.	AAATCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	67602	.	ATGTCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	68001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=70000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.57:1.36:.:0:.:0:0.999:152
+1	68303	.	CAGCTATTACCTATTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	68384	.	AGAGCTAAATTAAACAATCATTCAAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	68544	.	TTTTTCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	68613	.	ATATTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	68893	.	ATAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	68905	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	68931	.	GTAATAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	69060	.	GAATGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	69267	.	CTCACTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	69453	.	G	A	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2854682&dbsnp.134|rs142004627;CGA_FI=79501|NM_001005484.1|OR4F5|CDS|SYNONYMOUS;CGA_PFAM=PFAM|PF00001|7tm_1;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:PASS:129:129,129:129,129:42,45:-129,0,-129:-42,0,-45:28:4,24:24
+1	69511	.	A	G	.	.	NS=1;AN=2;AC=2;CGA_XR=dbsnp.100|rs2691305&dbsnp.131|rs75062661;CGA_FI=79501|NM_001005484.1|OR4F5|CDS|MISSENSE;CGA_PFAM=PFAM|PF00001|7tm_1;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/1:.:PASS:135:135,317:135,317:48,48:-317,-135,0:-48,-48,0:5:4,4:1
+1	69552	.	G	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2531266&dbsnp.129|rs55874132;CGA_FI=79501|NM_001005484.1|OR4F5|CDS|SYNONYMOUS;CGA_PFAM=PFAM|PF00001|7tm_1;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1|0:69552:PASS:130:130,178:130,178:42,49:-130,0,-178:-42,0,-49:23:4,19:19
+1	69569	.	T	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2531267;CGA_FI=79501|NM_001005484.1|OR4F5|CDS|MISSENSE;CGA_PFAM=PFAM|PF00001|7tm_1;CGA_SDO=2	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1|0:69552:PASS:177:177,178:177,178:48,49:-177,0,-178:-48,0,-49:29:8,21:21
+1	69732	.	CCTAATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	69894	.	TTCTGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	69965	.	GACAGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	70001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=72000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.24:1.30:.:0:.:0:0.999:152
+1	70242	.	GAACAAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	70297	.	TGACCCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	70349	.	TTTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	70496	.	ATACTTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	70604	.	G	A	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2854679;CGA_RPT=LTR89|ERVL?|39.7;CGA_SDO=3	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:VQLOW:28:28,28:28,28:6,25:-28,0,-28:-6,0,-25:16:2,14:14
+1	70725	.	AAGCACAGGCTTTAAAGTAAAAAACAAAGAGCTGGATTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	71176	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	71378	.	AATTTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	71632	.	AATATGGTAAAGATGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	71666	.	TTAATTTTTAATGCGTAATAAAACTATGAGAAAATTTAAAAGTGAGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	71779	.	CCCAAAATATTAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	71843	.	AGTCTTTTTTTTTTTTTTTACAGTTGTAGGCAGAAAACTTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	71989	.	AATTTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	72001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=74000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.86:1.18:.:0:.:0:0.999:152
+1	72082	.	TACTCTTTTATATATATACATATATGTGTGTATATGTGTATATATATATACACACATATATACATACATACATACATACATATTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	72206	.	GGGATACATGTGCAGAATGTACAGGTTTGTTACACAGGTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	72277	.	CAACTCACCATCTACATTAGGTATTTCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	72336	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	72389	.	TTGGTCAACTCCCATTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	72452	.	TGCGGAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	72520	.	GCTGCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	72585	.	TTAAAAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	72694	.	GTGTTTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	72743	.	CCATTAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	72787	.	C	T	.	.	NS=1;AN=2;AC=2;CGA_XR=dbsnp.100|rs2854675&dbsnp.129|rs62641289;CGA_RPT=L1PA7|L1|7.3;CGA_SDO=3	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/1:.:PASS:54:169,54:169,54:48,45:-169,-54,0:-48,-45,0:7:7,7:0
+1	72859	.	CCCGTCAATGTTAGACTAGATAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	73230	.	ATGTTTAGCTCCCCCTTGTTAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	73343	.	GACTTTCTTCTTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	73486	.	TGTCTTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	73620	.	C	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	73633	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	73645	.	TTCCCCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	73695	.	TATGAAAAAAATGTTCAAGTCTCTCAGATTAAGATGCATGCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	73799	.	TTTGATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	73838	.	CACCCTTTTTTTTTTTTTTAATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	73874	.	TAGGGTACATGTGCACCTTGTGCAGGTTAGTTACATATGTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	73932	.	TGCGCTGAACCCACTAACTCGTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	73971	.	ATCTCCCAATGCTATCCCTCCCCCCTCCCCCCACCCCACAACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74001	.	C	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=76000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.20:0.87:.:0:.:0:0.999:152
+1	74021	.	GAGTGTGATATTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74070	.	CCACCTATGAGTGAGAATATGCGGTGTTTGGTTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74112	.	T	<CGA_NOCALL>	.	.	END=74372;NS=1;AN=0	GT:PS	./.:.
+1	74384	.	TGGTATTTCCAGTTCGAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74408	.	GAGGAATCGCCACACTGACTTCCACAATGGTTGAACTAGTTTACAGTCCCACCAACAGTGTAAAAGTGTTCCTATTTCTCCACATCCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74503	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74506	.	TGTTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74520	.	TTTTTAATGAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74535	.	A	.	.	.	NS=1;AN=1	GT:PS:FT:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL	0|.:74535:VQLOW:28,.:23,.:0,.:0:0
+1	74550	.	AGATGATATCTCATTGTGGTTTTGATTTGCATTTCTCTGATGGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74610	.	TTTTTCATGTGTTTTTTGGCTGCATAGATGTCTTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74672	.	C	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74676	.	CACTTGTT	.	.	.	NS=1;AN=0	GT:PS	.|.:74676
+1	74690	.	GTTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74701	.	TTTTCTTGTAAATTTGTTTGAGTTCATTGTAGATTCTGGATATTAGCCCTTTGTCAGATGAGTAGGTTGCAAAAATTTTCTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74822	.	TTTTGCTGTGCAGAAGCTCTTTAGTTTAATTAGATCCCATTTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	74927	.	ATCAGAGAATACTACAAACACCTCTACGCAAATAAACTAGAAAATCTAGAAGAAATGGATAAATTCCTGGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	75004	.	CACTCTCCCAAGCCTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	75085	.	TAGCTTACCAACCAAAAAGAGTCCAGGACCAGATGGATTCACAGCCGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	75144	.	GGTAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	75172	.	TCTGAAACTATTCCAATCAATAGAAAAAGAGGGAGTCCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	75222	.	TTTATGAGGCCAGCATCATTCTGATACCAAAGCCAGGCAGAGACACAACAAAAAAAGAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	75301	.	GATGAACATTGATGCAAAAATCCTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	75333	.	TACTGGCAAAACGAATCCAGCAGCACATCAAAAAGCTTATCCACCAAGATCAAGTGGGCTTCATCCCTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	75409	.	AGGCTGGTTCAATATACGCAAATCAATAAATGTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	75451	.	TATAAACAGAGCCAAAGACAAAAACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	75490	.	AATAGATGCAGAAAAGGCCTTTGACAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	75523	.	A	<CGA_NOCALL>	.	.	END=75821;NS=1;AN=0	GT:PS	./.:.
+1	75828	.	CCATTGTCTCAGCCCAAAATCTTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	75876	.	A	<CGA_NOCALL>	.	.	END=76275;NS=1;AN=0	GT:PS	./.:.
+1	76001	.	C	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=78000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.60:0.90:.:0:.:0:0.999:152
+1	76285	.	AAAGAACAAAGCTGGAGGCATCACGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	76326	.	TATACGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	76416	.	TAATGCCGCATATCTACAACTATCTGATCTTTGACAAACCTGAGAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	76536	.	TGAAACTGGATCCCTTCCTTACACCTTATACAAAAATCAATTCAAGATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	76597	.	AAACGTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	76619	.	AAAACCCTAGAAGAAAACCTAGGCTTTACCATTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	76843	.	CACTCAAGTCTATTCATTGAAGCATTGTTTTTCATAGTAAACGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	76918	.	TGATCCCAGCATTTTGGGAGGCTGAGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	76992	.	TGGCGAAACCCCATCTCTACCAAAAATACAAAAATTAGCTGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	77081	.	GGGAGGATCGCTTGAACCTGGGAGGCAGAAGTTTCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	77129	.	CGTGCCTCTGCACTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	77175	.	AAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	77276	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	77409	.	ATAAGTATATATTTTATAAATGTTTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	77459	.	AACGTAATACATATATAATTTTCTTATGGCAGGAGGAGGAAACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	77565	.	ATCCTGTAGCTGTTTTATGTAATATAAAAATGTAATTAAATTAACAGAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	77688	.	CATGGGACACTAACATACAGACAAATTCATTTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	77773	.	CGAATAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	77916	.	TTGTTAAATATTCTCTATTTTATGACAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	77958	.	GTCGAAGAGAGAAACATGCAAGAACACCGTAGGGTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	78001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=80000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.80:1.04:.:0:.:0:0.999:152
+1	78015	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	78035	.	G	A	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2691297;CGA_RPT=L1MC4a|L1|35.4;CGA_SDO=3	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1|0:78035:VQLOW:23:23,23:23,23:2,24:-23,0,-23:-2,0,-24:8:1,7:7
+1	78093	.	TATATTTTTAAAAACTAAAAAGATATATTAGCTGATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	78193	.	TATTAAAATAATTTAAAAATGACCAAGTATTTGATTATATCAAATATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	78311	.	TAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	78354	.	GG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	78388	.	TGAAACCCTATCTCTACAAAAAACAAACAAAAATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	78437	.	TTTAAAAAATAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	78507	.	TGATCTGACTATGTGCTTCCCTGAACAAATGCACTTTACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	78630	.	CTTCTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	78892	.	CTCGCAGCCCTCACCCTGGAGAGTCCACAGGTACCAGGGGTTGGTCTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	78965	.	CACAGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	79022	.	CAAGCAGGGCCACCTGGCCTGGGACTCCGGTAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	79075	.	GACGACT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	79150	.	GCAGTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	79415	.	CCCGTGTCACAGCAGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	79663	.	A	G	.	.	NS=1;AN=2;AC=1;CGA_RPT=L1PREC2|L1|19.7;CGA_SDO=3	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1|0:79663:PASS:62:62,64:62,64:27,35:-62,0,-64:-27,0,-35:14:1,13:13
+1	79678	.	T	C	.	.	NS=1;AN=2;AC=1;CGA_RPT=L1PREC2|L1|19.7;CGA_SDO=3	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1|0:79663:PASS:54:54,64:54,64:24,35:-54,0,-64:-24,0,-35:17:1,16:16
+1	79769	.	CCTCCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	79876	.	AACGACCATACTGCCAAAAGCAACCTACAAATTCAATGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	80001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=82000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.15:1.21:.:0:.:0:0.999:152
+1	80141	.	A	G	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.108|rs4030287;CGA_RPT=L1PB|L1|9.0;CGA_SDO=3	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:PASS:106:106,106:106,106:26,33:-106,0,-106:-26,0,-33:32:7,25:25
+1	80383	.	AACCCTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	80500	.	AAATAAATAATCAGCAGAGTAAACAGACAACCCACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	81204	.	T	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.100|rs2531303;CGA_RPT=Tigger5|TcMar-Tigger|32.1;CGA_SDO=3	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:PASS:40:40,40:40,40:16,28:-40,0,-40:-16,0,-28:17:2,15:15
+1	81257	.	TGCCTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	81326	.	CAGAGACTGACTGTGTCAAAGTATTAGTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	81784	.	AAAATGTAAAAAGTATCTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	81856	.	TAATAAAATAAGAAGCCAAAAAACAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	82001	.	G	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=84000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.87:1.25:.:0:.:0:0.999:152
+1	82030	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	82112	.	ACAATTCACCACAACTGACTTCAAAAAAAAAAAAAAAAAAAAAGAAGTACCGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	82210	.	TATAACACACACACAAACACTAGGTTTAGATGTTTTCACAGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	82300	.	AACTCTCAGCCATTTGAGGCAAAATATTACAATTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	82453	.	GCCAAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	82537	.	AAATGAAGGCTAAGGCAGAATTATATATGGCTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	82620	.	AGAAGTTTTTCATATTTTTTTCTTTCTTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	82673	.	TTTTAAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	82734	.	T	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.108|rs4030331;CGA_RPT=L1M4c|L1|45.3;CGA_SDO=3	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:VQLOW:32:32,32:30,30:4,18:-32,0,-32:-4,0,-18:38:25,13:13
+1	82797	.	CATCCCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	82884	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	82961	.	CCCTGAGTAAGCAGATATTGAAAATATTAGACAAAAACTTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	83010	.	GTCTTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	83052	.	ACATATAAATAAATAAGAAATATGAATTTTTTTAAAGGTACAAAAAAATTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	83121	.	TAAGTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	83242	.	AATGACAACAAAAAAGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	83329	.	CAGGAAGACTATTTGAAGAAATGTGTTTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	83385	.	AATATAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	83412	.	ACTTCATCAAGGAAATATACAAAGATATTCACACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	83511	.	CAACGCG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	83556	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	83588	.	TTAGGAAAAAGGCAACGCGTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	83642	.	GAGAGCTCATTATAAACCATGGGTGCCAGAAGAGCTTAGGATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	83783	.	G	<CGA_NOCALL>	.	.	END=84056;NS=1;AN=0	GT:PS	./.:.
+1	84001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=86000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.21:1.10:.:0:.:0:0.999:152
+1	84091	.	CACTTTTAAAAAAAAGACTCCTTCAGATACAAACTAAAAAACACTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	84195	.	ATATAAAAGCAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	84241	.	TATATGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	84283	.	TATGTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	84299	.	GATGTATACAGATGTGGTTTGTGAAATTACCAACATAAAGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	84379	.	CTATTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	84435	.	AAATCCCCATGGTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	84585	.	ACAGAAAACAAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	84680	.	ATTAACAGAATGGATTTTTTAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	84750	.	ACACAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	85048	.	AAAAGATAAAACATCTAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	85118	.	CTATAGATAACACTTCTCTCAAAAACTGCAGAGTAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	85197	.	ATATGTTAGGCCATAAGATAAGCTCAATAAACTTAAAAAGATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	85403	.	AGACGATTGAAAACAAATATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	85490	.	AACATGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	85801	.	ATGCAAAAAAAATGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	85994	.	AAATGTACCAGAATCTGAAAACATCTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86001	.	C	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=88000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.57:0.68:.:0:.:0:0.999:152
+1	86300	.	TTAGTTCAAATTGACTTTTGAACATACTTGGACTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86417	.	AAATATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86429	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86435	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86448	.	C	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86452	.	GTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86469	.	AGTAAATATTAATATATTTGTATTGCTAGAACCCCAGAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86515	.	GTGAAAGGACAGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86573	.	TACATTAGAATCAGTATTATCAACATAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86636	.	TCTTAAAAAAATATAATATGGACATATTATATATTATATGCAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86687	.	GTGTGTCTATACAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86758	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86796	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86809	.	ATAATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86818	.	CCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	86983	.	ACATCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	88001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=90000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.60:0.82:.:0:.:0:0.999:152
+1	88065	.	CTAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	88125	.	TCTGACT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	88239	.	CCATGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	88259	.	GACCTTCTCCTGGGACCACAGGCCTGTGTCTCTATCTGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	88329	.	TAAAACT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	88570	.	TTTCACG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	88773	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	88797	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	88915	.	GCAGAGCCGGCCCCCATCTCCTCTGACCTCCCACCTCTCTCCCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	89251	.	AATATTGAGCACTATCAGTAAAATACATAAAACCCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	89346	.	CAAATGGATTACACGCAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	89412	.	TGGCTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	89575	.	TAATGAATAATTTTAATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	89628	.	GGTTTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	89845	.	AATACTCCCAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	89885	.	AGCCTCCATCTTTCCACTCCTTAATCTGGGCTTGGCCAAGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	89948	.	ATTAACAAGTCTGATGTGCACAGAGGCTGTAGAATGTGCACTGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	90001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=92000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.44:0.93:.:0:.:0:0.999:152
+1	90025	.	TGCCCCACGAAGGAAACAGAGCCAACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	90062	.	T	<CGA_NOCALL>	.	.	END=90300;NS=1;AN=0	GT:PS	./.:.
+1	90306	.	GACAGTCCCTCTGTCCCTCTGTCTCTGCCAACCAGTTAACCTGCTGCTTCCTGGAGGAAGACAGTCCCTCTGTCCCTCTGTCTCTGCCAACCAGTTAACCTGCTGCTTCCTGGAGGAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	90452	.	TTGACCGCAGACATGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	90638	.	TTCTCTGCTCATTTAAAATGCCCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	90674	.	TACATTTTTATAGGATCAGGGATCTGCTCTTGGATTTATGTCATGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	90777	.	AGGCGCTGGGAGGCCTGTGCATCAGCTGCTGCTGTCTGTAGCTGAGTTCCTTCACCCCTCTGCTGTCCTCAGCTCCTTCGCCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	90868	.	CAGGAAATCAATGTCATGCTGACATCACTCTAGATCTAAAACTTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	90939	.	ACATCTGTAATCCCAGCAATTTGGGAGGCCGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91004	.	GATCCTGGCTAACACGGTGAAACCCCGTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91061	.	GGTTTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91079	.	TGTAGCCCCAGCTACTTGGGAGGCTGAAGCAGGAGAATGGCGTGAACCTGGGAGGTGGAGCTGGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91160	.	GCCACTGCACTCCAGACTGGGAGAGAGAGCGAGACTTTCTCAAAAAAAAAAAAATCTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91252	.	CTAGAATCCTTGAAGCGCCCCCAAGGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91417	.	TGTGTGGCACCAGGTGGCAGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91450	.	GGCAAACCCGAGCCCAGGGATGCGGGGTGGGGGCAGGTACATCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91504	.	TACAGCAGATTAACTCTGTTCTGTTTCATTGTGGTTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91548	.	TGCGTTTTTTTTTCTCCAACTTTGTGCTTCATCGGGAAAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91613	.	GAAGAAAAGGCCAAACTCTGGAAAAAATTTGAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91666	.	GACCACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91727	.	AAAAGCCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91801	.	AGACAAAAAAGCTACATCCCTGCCTCTACCTCCATCGCATGCAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91886	.	AACCATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91937	.	TCCCCAATACCCGCAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	91983	.	CAACCTTTGGGAAAAGCAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	92001	.	C	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=94000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.09:1.17:.:0:.:0:0.999:152
+1	92143	.	GAACGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	92478	.	GCTGAGGCTGCTATTCTTTTGCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	92635	.	CCAAACCTCAGTCCCTCAGTTGTAAAATTAAAAAAAAAAAAAAGAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	92708	.	GATTTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	92833	.	GAGGGACAGAAACAAGTGGGAGAAGGTAAAGAGATGGACAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	92915	.	TATGTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	93120	.	GAAATACAGAAGAGAGATTTCTCATGGTTAAAACGAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	93279	.	C	T	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.111|rs4265376;CGA_RPT=L5|RTE|30.4;CGA_SDO=14	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:VQLOW:36:36,36:36,36:13,27:-36,0,-36:-13,0,-27:16:1,15:15
+1	93491	.	TCAATTTTATTGAAGTTCACTTCTGACCTCTTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	93662	.	TAGAGCTGAGACCATTTGCCACTCAGTTTCCTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	93725	.	CCGGTTTTTTTGTTTTTGTTTTTGTTTTTAGACGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	93804	.	GCTCACTGCAACCTCCGCTGCCTGTGTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	94001	.	G	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=96000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.97:1.22:.:0:.:0:0.999:152
+1	94011	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	94116	.	CTCGTAAGTAGATTACTACAATCACCTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	94221	.	AAATGAAAATCTGACCACGTTACTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	94257	.	TCCGCCTATGGCCGCTGTTAGGATCAAGTCTAAACTCCCGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	94461	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	94522	.	GCTGTGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	94567	.	GCCTCACTCAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	94722	.	AATCACATCACATTGCTTCCTTCATATTTTTTTGGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	94805	.	GCTCCTTTTCTTTTCTTTTCTTTTTTTTTTTTTTTTTTTTTTTGAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	94857	.	TCTCGCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	94890	.	TGCAATCTAGGCTCACTGCAAGCTCTGCCTCCTGGGTTCACGTCATTCTCCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	94975	.	ACCTACCACCACGCCTGGCTAATTTTTTTTTATTTTTTATTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	95147	.	GCATAAACTAAATGTTTTCCAAAGGGAATAGGGCAAAACAAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	95290	.	GGGCTCTCCACTTACAAGAAGAGAGCATGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	95412	.	CCTGTTAATTTAATCACACGGAACACTTCTATTTAAAATTCCCGAGAGTTAAGATGTAAGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	95532	.	CATCAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	95575	.	ATGGGGCAATTTCTTAAAAGCACCATGTATTTTATCG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	95695	.	CCACTATAAAGAACCCAGCGTGGTTTTAACTAATGGATCAAAAGATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	96001	.	T	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=98000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.96:1.21:.:0:.:0:0.999:152
+1	96005	.	TGGGAGGCACAGTGGAAGATCATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	96150	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	96250	.	TCACGGAGGAAAAAAATCTCTCAATGATCTTATCTTTATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	96363	.	GAGGCAACCTCCAAAGGTGGGGCCCTCTGCTCACCTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	96476	.	TTTCTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	96589	.	AAGTACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	96636	.	ATAAATTCGTTCAAGCAGCCATTCTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	96895	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	96941	.	CGGTAGACTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	97044	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	97127	.	TTATAAAAAGGTGAGCTGTAATAAATACTAGTGCCACATTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	97279	.	AAGTACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	97311	.	ACCGGCAAATTCTGTTGTTTGTATAAACATCAGCCATGTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	97461	.	ATAATTAATACATTATTAAATTGAATTGTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	97563	.	TCAAATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	97621	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	97624	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	97818	.	AGACATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	97866	.	CCACGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	97929	.	TAACAATCTGAGAGACATTCATACATTTTCCATGTGCTGTAGCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98001	.	C	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=100000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.74:1.17:.:0:.:0:0.999:152
+1	98018	.	CCCTGTTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98103	.	AATAAAGAATTCTATCAATGCTGAGGGAAGATGACTAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98307	.	CCTGCCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98378	.	T	C	.	.	NS=1;AN=2;AC=1;CGA_XR=dbsnp.108|rs3868675&dbsnp.108|rs4114931;CGA_SDO=14	GT:PS:FT:GQ:HQ:EHQ:CGA_CEHQ:GL:CGA_CEGL:DP:AD:CGA_RDP	1/0:.:VQLOW:37:37,37:37,37:14,27:-37,0,-37:-14,0,-27:17:1,16:16
+1	98447	.	ACATGGGCACCCATATTTTTCTAGCCACTTTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98507	.	C	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98566	.	GTGATTTTCTGTTGGTGTTCACTTCAGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98615	.	TTATTGACTGACTGACTAACTAATGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98652	.	TTCATAAAGAAAGGCTCTCTACAAAAACGGAGGGATGCCCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98776	.	AATGTGCCTTTCTAGTAACAGGTTTTTAGAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98830	.	TATTTGTGTGTGTGCATGTGGTAGTGGGGAGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98882	.	AGAAAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98910	.	ATACTGTATTCAGGGGGAAAAAATTTTCCCAAGGTCCTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	98996	.	TGCTTTTATTTATTTATTTATTTATTTATTTATTTATTTATTTATTTTTCCTTTTTTTTCTTTCTCTTTTTTTCTTCTTTTTTTTTTCTTTTCTTTCTTTTTTTTTTTTTTTTTTTTTTTTGGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	99331	.	CTCATGATCCACCCACGTTGGCCTCCCAAAGTGCTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	99373	.	CAGGCGTGAGCCACCGCCCCTGGCCAGGATTGCTTTTACAGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	99480	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	99488	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	99497	.	C	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	99584	.	GTTAAAAGATATTATTTTGCTTTACACTTTTTCTCTCAGAAATAAGCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	99754	.	AGGGGAGATTTTTCAGGAGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	99820	.	GAAAGTGTATAATGATGTCAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	99871	.	TGCCGTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	99938	.	AAAAGTTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	100001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=102000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.83:1.22:.:0:.:0:0.999:153
+1	100319	.	TTTGGACAGTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	100481	.	TCCGTGTTACTGAGCAGTTCTCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	100538	.	TTCTTTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	100583	.	AGGTCAAATTCAAAGGAGAGAAAAAAGCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	100652	.	ATGGCACAATTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	100750	.	TACCCTTCTAATCTCTATCACAGCAAAAAGATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	100855	.	ACATTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	100942	.	AATGTAGAAATGCTACAGATTATATTCTCTGATTATGACACAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101011	.	TTTAAAAGCTTTCTCTTAAATAATTCTATGTCAAAAAGAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101092	.	CTTTGGGAGGCCAAGGTGGGCAGGTCACTTGAGGTCAGCAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101140	.	CCAGCCTCGTCAACATGGTAACACCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101210	.	TGCCTGTAATCCCAGCTACTTAGGAGGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101265	.	AAGGTGGAGGTTGCAGTGAGCTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101312	.	TAGGTGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101376	.	GTGGAAAATAGTGACAATAAAAATATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101425	.	TTGAGATGCCAAGGTGGCAGGATCACTTGAGACCAGGAGTTCGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101509	.	C	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101512	.	C	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101547	.	TTCCTGTAATCCCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101608	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101618	.	C	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101666	.	TCCTATCTCAAAAAAAAAAAAAAAATCAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101792	.	GGAAATAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101825	.	TAGCCACGGTGACTCACATCTGTAATCCCAGCACTTTGGGAGGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	101909	.	GCCTGGCCAACATGGTGAAATCTTGTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	102001	.	G	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=104000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:0.86:1.32:.:0:.:0:0.999:153
+1	102133	.	ATAAACTAGAAAACAGAAACATAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	102181	.	ATGCCTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	102207	.	GTGAATTAAGGAAGGGAAGAGATGGTTGGAGAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	102322	.	AGAGATGCTTGACTGCTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	102412	.	ATCTCCTCCCCTCCCCTACTCCTCACCCCACACTCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	102494	.	GCATTCTTATTTCCCTGATTTCTTTTTGAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	102593	.	CAAGGGCTTCACAGACAGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	102635	.	TTCAGGTTTTATACCTACCTTATAGATAAAGAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	102693	.	TGTTCCCAAAGCCTCGTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	102766	.	ACTCTACTGCCTCTCCATGGATAAAGACAGAGATCACATATTAATA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	102856	.	TGATTAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	102915	.	AATTTGAATAACTCCCTGCGGGTGAAGTTCAAAGTACTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103007	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103088	.	TCTACTAAAAATAAAAAATTAGCCGGGCCTGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103165	.	G	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103214	.	CACTGCACTCCAGCCTGGGCGACAGAGCGAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103257	.	AAAGTAAAATAAAATAAAATAAAAAATAAAAGTTTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103304	.	ATCAGGGAGGTCTGTTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103336	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103381	.	TCAGGGTCCTAGCAGGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103441	.	AGGGTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103523	.	TGTTGGAGGTGGGGCCTAATGGGAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103737	.	CAGAGTAGCTAGGATTACAGGTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103800	.	AGACGGGGTTTCACCAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103853	.	GATACACCTGCCTCGGCCTCCCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103890	.	CAGGTGTGAGCCACCATGCCTACC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	103970	.	AACCCCTCTCTCTCGCCACGTGATCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	104001	.	C	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=106000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.06:1.45:.:0:.:0:0.999:153
+1	104030	.	GAGTGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	104104	.	AGCCAAATAAACCTCTCTTCTTTAAAATTATTCAGCCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	104157	.	AACAACACACACACACACACACACACACATACACACACACG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	104222	.	AATTAGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	104317	.	TAATGGTTAAGTAATTATTTGCTCTTACTCTCAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	104382	.	TCAACTAGAATCTAGGAAGCAGAGAACCTGAGTGTTGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	104519	.	CAACAGAGCGACTCAGATGCTATAAAACTTGCTAACA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	104611	.	CACAGTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	104672	.	ACCTCACAGAGAAGGAAATTTACACGCGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	104756	.	TTCTGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	104819	.	GACATTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	104939	.	ATACAAAGAGTAATACCATGTCACTTAAGAATAGAATCATGGACGAGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	105054	.	TTCCTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	105120	.	ACACAAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	105178	.	GCTCAGATACCTTCTCCGCTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	105265	.	GAGACTAATGAGTAGTGAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	105295	.	AAGCTGAGAATGCTTCTACCTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	105330	.	GGAATATTCATCAAAACACAGCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	105484	.	TTCCAGAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	105631	.	ACACTCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	105713	.	TACAATAAACATGTGTTTTTAACAAGAAAAGTCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	105785	.	A	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	105886	.	TAGAGGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	105984	.	ACATGGGTGTTAAAATCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	106001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=108000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.19:1.25:.:0:.:0:0.999:153
+1	106040	.	AGAGCAAGCTGGGAAAGCAGTGGCCTTTAATAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	106115	.	C	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	106124	.	AGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	106130	.	T	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	106133	.	TAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	106154	.	AATAGTAAACTGAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	106462	.	AGAATAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	106497	.	GTAATTTTAATATATAACTGGGGTGAGAATCATTGACATAATTGTAACAGGATAATATTCAGGAAATATGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	106608	.	AAAAGTTTTATGTTTTCCCCTAACTCAGGGTCATCAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	106908	.	CTTGAGCAAATGGTAAATTAACTCTCTCTTTTCTCTCTCTCTCTAGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	107122	.	TTTACTGGAGTACACAATTGTGACTATTTTTAGCCATAGGAAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	107279	.	CACCTTACACTTAGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	107329	.	CACTTTTCAAAAGACT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	107514	.	GGAGATTTGGACATAGAGAGAGGCACACGGGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	107967	.	TGGGCTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	108001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=110000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.13:1.11:.:0:.:0:0.999:153
+1	108016	.	TCTTTCCTGGCTATGTTTCTGACATCCTCTTGTACCATGCTCCTCC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	108252	.	CTATAACAACCTAATATATTCTCAATTGATTAACTGTTTTGCTGAATAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	108318	.	GAAAGAAAACATGGCCAGGTGCAGTGGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	108356	.	AATCCCACCACTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	108401	.	CTTCAAAAAATTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	108537	.	GTCACTATCAAAAAAAAAAAAAAAAGAAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	108596	.	TATCTAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	108672	.	AATGCAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	108890	.	TGACCCAGCATGGCTGAACACTCAGTGACTACCAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	108998	.	TTATATTCAGAATTACTCAAGTCTTAGAAGCACCACTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	109061	.	TCAAGTGATGGGCTGAAGTGAAGGGAGGGAGTCACTCACTTGAACGGT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	109403	.	ATATTTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	109568	.	GTGTATGCGTGTGTGTGTGTGTGTGTGTGTGTGTGTGTGTGTGTGTGAAAGACAGAAGAAAGAGGGAGACCTTAGAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	109707	.	AGGGAGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	109773	.	ATATATGCAATATATATACATATATACACACATATACATATGTATTTAAATATTTAAATTACATTTTCTCTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	109852	.	AGATATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	109951	.	CAACCCTCCTGTATTAGTCTCCCCAGTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	110001	.	A	<CGA_CNVWIN>	.	.	NS=1;CGA_WINEND=112000	GT:CGA_GP:CGA_NP:CGA_CP:CGA_PS:CGA_CT:CGA_TS:CGA_CL:CGA_LS	.:1.05:1.31:.:0:.:0:0.999:153
+1	110004	.	ATGTCCACCTTTATGCTTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	110059	.	AACTTAATAATAAAAACATTTCAAATGTAAAGAAATTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	110124	.	AAATTGG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	110148	.	ACACTTTTCAAAAGAATACATGCATGCA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	110217	.	TTAGAGAAATGCAAATCAAAACCATAATGAGATACCATCTC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	110416	.	CCATTCG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	110438	.	TACTGGGTATATACCCAGATGAATATAAACCAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	110504	.	TTGCAGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	110606	.	ATGCCGC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	110700	.	ATACAGCATACTCTCAGTTATAAGTGGGAGCTAAATGAT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	110925	.	AAATAAAAGTTAAAAAAAAAAGAAAATTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	110980	.	TATGAAAAACACATATCTTTCATT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	111099	.	ACACCTGTAATCCCAGCACTTTGGGAGGCCGATG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	111158	.	TTCGGGA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	111201	.	TAAAAATACAAAAATTAGCTGGGTGTGGTGGCAGGCACCTGTAATAC	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	111259	.	GAGGCTGAGGCAGGAGAATCGCTT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	111341	.	GCAACAA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	111363	.	GGGGAAAAAAAAAAACAAAAAAAACCACCACCATCATTTTGCAAG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	111520	.	ATTATTTTGTATGCGATGACAACAGAATATATTATCATGCT	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	111570	.	AATCTCATTCATAATATAAAGTATAAATTTGTGATTTTG	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	111708	.	AAAATTTGAAACTAGTAACATGGAGGACTATTGTCATTGTTTA	.	.	.	NS=1;AN=0	GT:PS	./.:.
+1	111779	.	CAGTGTACA

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/beam/blob/f22da33c/sdks/python/apache_beam/testing/data/vcf/valid-4.1-large.vcf.gz
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/testing/data/vcf/valid-4.1-large.vcf.gz b/sdks/python/apache_beam/testing/data/vcf/valid-4.1-large.vcf.gz
new file mode 100644
index 0000000..e2d0fed
Binary files /dev/null and b/sdks/python/apache_beam/testing/data/vcf/valid-4.1-large.vcf.gz differ


[4/4] beam git commit: This closes #3979

Posted by ch...@apache.org.
This closes #3979


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/0af97209
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/0af97209
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/0af97209

Branch: refs/heads/master
Commit: 0af972095a9a233831dd98cb8b4057f8b2f25dfe
Parents: 9c54542 f22da33
Author: chamikara@google.com <ch...@google.com>
Authored: Wed Nov 8 08:47:44 2017 -0800
Committer: chamikara@google.com <ch...@google.com>
Committed: Wed Nov 8 08:47:44 2017 -0800

----------------------------------------------------------------------
 pom.xml                                         |     3 +
 sdks/python/apache_beam/io/source_test_utils.py |     3 +-
 sdks/python/apache_beam/io/textio.py            |    67 +-
 sdks/python/apache_beam/io/textio_test.py       |   503 +-
 sdks/python/apache_beam/io/vcfio.py             |   436 +
 sdks/python/apache_beam/io/vcfio_test.py        |   519 +
 .../apache_beam/testing/data/vcf/valid-4.0.vcf  |    23 +
 .../testing/data/vcf/valid-4.0.vcf.bz2          |   Bin 0 -> 781 bytes
 .../testing/data/vcf/valid-4.0.vcf.gz           |   Bin 0 -> 727 bytes
 .../testing/data/vcf/valid-4.1-large.vcf        | 10000 +++++++++++++++++
 .../testing/data/vcf/valid-4.1-large.vcf.gz     |   Bin 0 -> 156715 bytes
 .../apache_beam/testing/data/vcf/valid-4.2.vcf  |    42 +
 .../testing/data/vcf/valid-4.2.vcf.gz           |   Bin 0 -> 1240 bytes
 sdks/python/apache_beam/testing/test_utils.py   |    40 +
 .../apache_beam/testing/test_utils_test.py      |    24 +
 sdks/python/generate_pydoc.sh                   |     6 +
 sdks/python/setup.py                            |     3 +-
 17 files changed, 11419 insertions(+), 250 deletions(-)
----------------------------------------------------------------------