-
Notifications
You must be signed in to change notification settings - Fork 4.3k
/
Copy pathmongodbio_it_test.py
170 lines (154 loc) · 5.34 KB
/
mongodbio_it_test.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
#
# 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.
#
# pytype: skip-file
import argparse
import logging
import time
from pymongo import MongoClient
import apache_beam as beam
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.testing.test_pipeline import TestPipeline
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
_LOGGER = logging.getLogger(__name__)
class GenerateDocs(beam.DoFn):
def process(self, num_docs, *args, **kwargs):
for i in range(num_docs):
yield {'number': i, 'number_mod_2': i % 2, 'number_mod_3': i % 3}
def run(argv=None):
default_db = 'beam_mongodbio_it_db'
default_coll = 'integration_test_%d' % time.time()
parser = argparse.ArgumentParser()
parser.add_argument(
'--mongo_uri',
default='mongodb://localhost:27017',
help='mongo uri string for connection')
parser.add_argument(
'--mongo_db', default=default_db, help='mongo uri string for connection')
parser.add_argument(
'--mongo_coll',
default=default_coll,
help='mongo uri string for connection')
parser.add_argument(
'--num_documents',
default=100000,
help='The expected number of documents to be generated '
'for write or read',
type=int)
parser.add_argument(
'--batch_size',
default=10000,
type=int,
help=('batch size for writing to mongodb'))
known_args, pipeline_args = parser.parse_known_args(argv)
# Test Write to MongoDB
with TestPipeline(options=PipelineOptions(pipeline_args)) as p:
start_time = time.time()
_LOGGER.info('Writing %d documents to mongodb', known_args.num_documents)
_ = (
p | beam.Create([known_args.num_documents])
| 'Create documents' >> beam.ParDo(GenerateDocs())
| 'WriteToMongoDB' >> beam.io.WriteToMongoDB(
known_args.mongo_uri,
known_args.mongo_db,
known_args.mongo_coll,
known_args.batch_size))
elapsed = time.time() - start_time
_LOGGER.info(
'Writing %d documents to mongodb finished in %.3f seconds' %
(known_args.num_documents, elapsed))
# Test Read from MongoDB
total_sum = sum(range(known_args.num_documents))
mod_3_sum = sum(
num for num in range(known_args.num_documents) if num % 3 == 0)
mod_3_count = sum(
1 for num in range(known_args.num_documents) if num % 3 == 0)
# yapf: disable
read_cases = [
# (reader_params, expected)
(
{
'projection': ['number']
},
{
'number_sum': total_sum,
'docs_count': known_args.num_documents
}
),
(
{
'filter': {'number_mod_3': 0},
'projection': ['number']
},
{
'number_sum': mod_3_sum,
'docs_count': mod_3_count
}
),
(
{
'projection': ['number'],
'bucket_auto': True
},
{
'number_sum': total_sum,
'docs_count': known_args.num_documents
}
),
(
{
'filter': {'number_mod_3': 0},
'projection': ['number'],
'bucket_auto': True
},
{
'number_sum': mod_3_sum,
'docs_count': mod_3_count
}
),
]
# yapf: enable
for reader_params, expected in read_cases:
with TestPipeline(options=PipelineOptions(pipeline_args)) as p:
start_time = time.time()
_LOGGER.info('=' * 80)
_LOGGER.info(
'Reading from mongodb %s:%s',
known_args.mongo_db,
known_args.mongo_coll)
_LOGGER.info('reader params : %s', reader_params)
_LOGGER.info('expected results: %s', expected)
docs = (
p | 'ReadFromMongoDB' >> beam.io.ReadFromMongoDB(
known_args.mongo_uri,
known_args.mongo_db,
known_args.mongo_coll,
**reader_params)
| 'Map' >> beam.Map(lambda doc: doc['number']))
number_sum = (docs | 'Combine' >> beam.CombineGlobally(sum))
docs_count = (docs | 'Count' >> beam.combiners.Count.Globally())
r = ([number_sum, docs_count] | 'Flatten' >> beam.Flatten())
assert_that(r, equal_to([expected['number_sum'], expected['docs_count']]))
elapsed = time.time() - start_time
_LOGGER.info(
'Reading documents from mongodb finished in %.3f seconds', elapsed)
# Clean-up
with MongoClient(host=known_args.mongo_uri) as client:
client.drop_database(known_args.mongo_db)
if __name__ == "__main__":
logging.getLogger().setLevel(logging.INFO)
run()