Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 5159130

Browse files
authored
Merge pull request #10029 from kkucharc/BEAM-6335-consume-data-insertion-pipeline-group-by-key
[BEAM-6335] Test GBK streaming reading SyntheticSources
2 parents e01f718 + 2c89d7f commit 5159130

9 files changed

Lines changed: 359 additions & 45 deletions

File tree

sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SyntheticDataPublisher.java

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -26,9 +26,9 @@
2626
import java.util.HashMap;
2727
import java.util.Map;
2828
import org.apache.beam.sdk.Pipeline;
29-
import org.apache.beam.sdk.coders.ByteArrayCoder;
29+
import org.apache.beam.sdk.coders.Coder;
3030
import org.apache.beam.sdk.coders.CoderException;
31-
import org.apache.beam.sdk.coders.KvCoder;
31+
import org.apache.beam.sdk.coders.StringUtf8Coder;
3232
import org.apache.beam.sdk.io.Read;
3333
import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
3434
import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
@@ -73,8 +73,7 @@
7373
*/
7474
public class SyntheticDataPublisher {
7575

76-
private static final KvCoder<byte[], byte[]> RECORD_CODER =
77-
KvCoder.of(ByteArrayCoder.of(), ByteArrayCoder.of());
76+
private static final Coder RECORD_CODER = StringUtf8Coder.of();
7877

7978
private static Options options;
8079

@@ -215,7 +214,7 @@ public PubsubMessage apply(KV<byte[], byte[]> input) {
215214

216215
private static byte[] encodeInputElement(KV<byte[], byte[]> input) {
217216
try {
218-
return encodeToByteArray(RECORD_CODER, input);
217+
return encodeToByteArray(RECORD_CODER, new String(input.getValue(), UTF_8));
219218
} catch (CoderException e) {
220219
throw new RuntimeException(String.format("Couldn't encode element. Exception: %s", e));
221220
}

sdks/python/apache_beam/io/gcp/tests/pubsub_matcher.py

Lines changed: 25 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -49,9 +49,9 @@ class PubSubMessageMatcher(BaseMatcher):
4949
subscription until all expected messages are shown or timeout.
5050
"""
5151

52-
def __init__(self, project, sub_name, expected_msg,
53-
timeout=DEFAULT_TIMEOUT, with_attributes=False,
54-
strip_attributes=None):
52+
def __init__(self, project, sub_name, expected_msg=None,
53+
expected_msg_len=None, timeout=DEFAULT_TIMEOUT,
54+
with_attributes=False, strip_attributes=None):
5555
"""Initialize PubSubMessageMatcher object.
5656
5757
Args:
@@ -75,22 +75,31 @@ def __init__(self, project, sub_name, expected_msg,
7575
raise ValueError('Invalid project %s.' % project)
7676
if not sub_name:
7777
raise ValueError('Invalid subscription %s.' % sub_name)
78-
if not isinstance(expected_msg, list):
78+
if not expected_msg_len and not expected_msg:
79+
raise ValueError('Required expected_msg: {} or expected_msg_len: {}.'
80+
.format(expected_msg, expected_msg_len))
81+
if expected_msg and not isinstance(expected_msg, list):
7982
raise ValueError('Invalid expected messages %s.' % expected_msg)
83+
if expected_msg_len and not isinstance(expected_msg_len, int):
84+
raise ValueError('Invalid expected messages %s.' % expected_msg_len)
8085

8186
self.project = project
8287
self.sub_name = sub_name
8388
self.expected_msg = expected_msg
89+
self.expected_msg_len = expected_msg_len or len(self.expected_msg)
8490
self.timeout = timeout
8591
self.messages = None
8692
self.with_attributes = with_attributes
8793
self.strip_attributes = strip_attributes
8894

8995
def _matches(self, _):
9096
if self.messages is None:
91-
self.messages = self._wait_for_messages(len(self.expected_msg),
97+
self.messages = self._wait_for_messages(self.expected_msg_len,
9298
self.timeout)
93-
return Counter(self.messages) == Counter(self.expected_msg)
99+
if self.expected_msg:
100+
return Counter(self.messages) == Counter(self.expected_msg)
101+
else:
102+
return len(self.messages) == self.expected_msg_len
94103

95104
def _wait_for_messages(self, expected_num, timeout):
96105
"""Wait for messages from given subscription."""
@@ -131,18 +140,21 @@ def _wait_for_messages(self, expected_num, timeout):
131140

132141
def describe_to(self, description):
133142
description.append_text(
134-
'Expected %d messages.' % len(self.expected_msg))
143+
'Expected %d messages.' % self.expected_msg_len)
135144

136145
def describe_mismatch(self, _, mismatch_description):
137146
c_expected = Counter(self.expected_msg)
138147
c_actual = Counter(self.messages)
139148
mismatch_description.append_text(
140-
"Got %d messages. "
141-
"Diffs (item, count):\n"
142-
" Expected but not in actual: %s\n"
143-
" Unexpected: %s" % (
144-
len(self.messages), (c_expected - c_actual).items(),
145-
(c_actual - c_expected).items()))
149+
"Got %d messages. " % (
150+
len(self.messages)))
151+
if self.expected_msg:
152+
mismatch_description.append_text(
153+
"Diffs (item, count):\n"
154+
" Expected but not in actual: %s\n"
155+
" Unexpected: %s" % (
156+
(c_expected - c_actual).items(),
157+
(c_actual - c_expected).items()))
146158
if self.with_attributes and self.strip_attributes:
147159
mismatch_description.append_text(
148160
'\n Stripped attributes: %r' % self.strip_attributes)

sdks/python/apache_beam/io/gcp/tests/pubsub_matcher_test.py

Lines changed: 53 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,6 @@
3333
from apache_beam.testing.test_utils import PullResponseMessage
3434
from apache_beam.testing.test_utils import create_pull_response
3535

36-
# Protect against environments where pubsub library is not available.
3736
try:
3837
from google.cloud import pubsub
3938
except ImportError:
@@ -54,14 +53,18 @@ def setUpClass(cls):
5453
def setUp(self):
5554
self.mock_presult = mock.MagicMock()
5655

57-
def init_matcher(self, with_attributes=False, strip_attributes=None):
56+
def init_matcher(self, expected_msg=None,
57+
with_attributes=False, strip_attributes=None):
5858
self.pubsub_matcher = PubSubMessageMatcher(
59-
'mock_project', 'mock_sub_name', ['mock_expected_msg'],
59+
'mock_project', 'mock_sub_name', expected_msg,
6060
with_attributes=with_attributes, strip_attributes=strip_attributes)
6161

62+
def init_counter_matcher(self, expected_msg_len=1):
63+
self.pubsub_matcher = PubSubMessageMatcher(
64+
'mock_project', 'mock_sub_name', expected_msg_len=expected_msg_len)
65+
6266
def test_message_matcher_success(self, mock_get_sub, unsued_mock):
63-
self.init_matcher()
64-
self.pubsub_matcher.expected_msg = [b'a', b'b']
67+
self.init_matcher(expected_msg=[b'a', b'b'])
6568
mock_sub = mock_get_sub.return_value
6669
mock_sub.pull.side_effect = [
6770
create_pull_response([PullResponseMessage(b'a', {})]),
@@ -72,8 +75,8 @@ def test_message_matcher_success(self, mock_get_sub, unsued_mock):
7275
self.assertEqual(mock_sub.acknowledge.call_count, 2)
7376

7477
def test_message_matcher_attributes_success(self, mock_get_sub, unsued_mock):
75-
self.init_matcher(with_attributes=True)
76-
self.pubsub_matcher.expected_msg = [PubsubMessage(b'a', {'k': 'v'})]
78+
self.init_matcher(expected_msg=[PubsubMessage(b'a', {'k': 'v'})],
79+
with_attributes=True)
7780
mock_sub = mock_get_sub.return_value
7881
mock_sub.pull.side_effect = [
7982
create_pull_response([PullResponseMessage(b'a', {'k': 'v'})])
@@ -83,8 +86,8 @@ def test_message_matcher_attributes_success(self, mock_get_sub, unsued_mock):
8386
self.assertEqual(mock_sub.acknowledge.call_count, 1)
8487

8588
def test_message_matcher_attributes_fail(self, mock_get_sub, unsued_mock):
86-
self.init_matcher(with_attributes=True)
87-
self.pubsub_matcher.expected_msg = [PubsubMessage(b'a', {})]
89+
self.init_matcher(expected_msg=[PubsubMessage(b'a', {})],
90+
with_attributes=True)
8891
mock_sub = mock_get_sub.return_value
8992
# Unexpected attribute 'k'.
9093
mock_sub.pull.side_effect = [
@@ -96,9 +99,9 @@ def test_message_matcher_attributes_fail(self, mock_get_sub, unsued_mock):
9699
self.assertEqual(mock_sub.acknowledge.call_count, 1)
97100

98101
def test_message_matcher_strip_success(self, mock_get_sub, unsued_mock):
99-
self.init_matcher(with_attributes=True,
102+
self.init_matcher(expected_msg=[PubsubMessage(b'a', {'k': 'v'})],
103+
with_attributes=True,
100104
strip_attributes=['id', 'timestamp'])
101-
self.pubsub_matcher.expected_msg = [PubsubMessage(b'a', {'k': 'v'})]
102105
mock_sub = mock_get_sub.return_value
103106
mock_sub.pull.side_effect = [create_pull_response([
104107
PullResponseMessage(b'a', {'id': 'foo', 'timestamp': 'bar', 'k': 'v'})
@@ -108,9 +111,9 @@ def test_message_matcher_strip_success(self, mock_get_sub, unsued_mock):
108111
self.assertEqual(mock_sub.acknowledge.call_count, 1)
109112

110113
def test_message_matcher_strip_fail(self, mock_get_sub, unsued_mock):
111-
self.init_matcher(with_attributes=True,
114+
self.init_matcher(expected_msg=[PubsubMessage(b'a', {'k': 'v'})],
115+
with_attributes=True,
112116
strip_attributes=['id', 'timestamp'])
113-
self.pubsub_matcher.expected_msg = [PubsubMessage(b'a', {'k': 'v'})]
114117
mock_sub = mock_get_sub.return_value
115118
# Message is missing attribute 'timestamp'.
116119
mock_sub.pull.side_effect = [create_pull_response([
@@ -122,8 +125,7 @@ def test_message_matcher_strip_fail(self, mock_get_sub, unsued_mock):
122125
self.assertEqual(mock_sub.acknowledge.call_count, 1)
123126

124127
def test_message_matcher_mismatch(self, mock_get_sub, unused_mock):
125-
self.init_matcher()
126-
self.pubsub_matcher.expected_msg = [b'a']
128+
self.init_matcher(expected_msg=[b'a'])
127129
mock_sub = mock_get_sub.return_value
128130
mock_sub.pull.side_effect = [
129131
create_pull_response([PullResponseMessage(b'c', {}),
@@ -140,7 +142,7 @@ def test_message_matcher_mismatch(self, mock_get_sub, unused_mock):
140142
self.assertEqual(mock_sub.acknowledge.call_count, 1)
141143

142144
def test_message_matcher_timeout(self, mock_get_sub, unused_mock):
143-
self.init_matcher()
145+
self.init_matcher(expected_msg=[b'a'])
144146
mock_sub = mock_get_sub.return_value
145147
mock_sub.return_value.full_name.return_value = 'mock_sub'
146148
self.pubsub_matcher.timeout = 0.1
@@ -149,6 +151,41 @@ def test_message_matcher_timeout(self, mock_get_sub, unused_mock):
149151
self.assertTrue(mock_sub.pull.called)
150152
self.assertEqual(mock_sub.acknowledge.call_count, 0)
151153

154+
def test_message_count_matcher_below_fail(self, mock_get_sub, unused_mock):
155+
self.init_counter_matcher(expected_msg_len=1)
156+
mock_sub = mock_get_sub.return_value
157+
mock_sub.pull.side_effect = [
158+
create_pull_response([PullResponseMessage(b'c', {}),
159+
PullResponseMessage(b'd', {})]),
160+
]
161+
with self.assertRaises(AssertionError) as error:
162+
hc_assert_that(self.mock_presult, self.pubsub_matcher)
163+
self.assertEqual(mock_sub.pull.call_count, 1)
164+
self.assertTrue(
165+
'\nExpected: Expected 1 messages.\n but: Got 2 messages.'
166+
in str(error.exception.args[0]))
167+
168+
def test_message_count_matcher_above_fail(self, mock_get_sub, unused_mock):
169+
self.init_counter_matcher(expected_msg_len=1)
170+
mock_sub = mock_get_sub.return_value
171+
self.pubsub_matcher.timeout = 0.1
172+
with self.assertRaisesRegex(AssertionError, r'Expected 1.*\n.*Got 0'):
173+
hc_assert_that(self.mock_presult, self.pubsub_matcher)
174+
self.assertTrue(mock_sub.pull.called)
175+
self.assertEqual(mock_sub.acknowledge.call_count, 0)
176+
177+
def test_message_count_matcher_success(self, mock_get_sub, unused_mock):
178+
self.init_counter_matcher(expected_msg_len=15)
179+
mock_sub = mock_get_sub.return_value
180+
mock_sub.pull.side_effect = [create_pull_response(
181+
[PullResponseMessage(
182+
b'a', {'foo': 'bar'})
183+
for _ in range(15)]
184+
)]
185+
hc_assert_that(self.mock_presult, self.pubsub_matcher)
186+
self.assertEqual(mock_sub.pull.call_count, 1)
187+
self.assertEqual(mock_sub.acknowledge.call_count, 1)
188+
152189

153190
if __name__ == '__main__':
154191
logging.getLogger().setLevel(logging.INFO)

sdks/python/apache_beam/testing/load_tests/load_test.py

Lines changed: 9 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -47,9 +47,10 @@ def parseTestPipelineOptions(self, options=None):
4747
)
4848
}
4949

50-
def setUp(self):
51-
self.pipeline = TestPipeline()
52-
self.input_options = json.loads(self.pipeline.get_option('input_options'))
50+
def setUp(self, pipeline_options=None):
51+
self.pipeline = TestPipeline(pipeline_options)
52+
input = self.pipeline.get_option('input_options') or '{}'
53+
self.input_options = json.loads(input)
5354
self.project_id = self.pipeline.get_option('project')
5455

5556
self.metrics_dataset = self.pipeline.get_option('metrics_dataset')
@@ -66,10 +67,12 @@ def setUp(self):
6667
)
6768

6869
def tearDown(self):
69-
result = self.pipeline.run()
70-
result.wait_until_finish()
70+
if not hasattr(self, 'result'):
71+
self.result = self.pipeline.run()
72+
self.result.wait_until_finish()
7173

72-
self.metrics_monitor.publish_metrics(result)
74+
if self.metrics_monitor:
75+
self.metrics_monitor.publish_metrics(self.result)
7376

7477
def get_option_or_default(self, opt_name, default=0):
7578
"""Returns a pipeline option or a default value if it was not provided.

sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -201,7 +201,7 @@ def publish_metrics(self, result):
201201
# required to prepare metrics for publishing purposes. Expected is to have
202202
# a list of dictionaries matching the schema.
203203
insert_dicts = self._prepare_all_metrics(metrics)
204-
if len(insert_dicts):
204+
if len(insert_dicts) > 0:
205205
for publisher in self.publishers:
206206
publisher.publish(insert_dicts)
207207

@@ -224,10 +224,11 @@ def _get_distributions(self, distributions, metric_id):
224224
matching_namsespace, not_matching_namespace = \
225225
split_metrics_by_namespace_and_name(distributions, self._namespace,
226226
RUNTIME_METRIC)
227-
runtime_metric = RuntimeMetric(matching_namsespace, metric_id)
228-
rows.append(runtime_metric.as_dict())
229-
230-
rows += get_generic_distributions(not_matching_namespace, metric_id)
227+
if len(matching_namsespace) > 0:
228+
runtime_metric = RuntimeMetric(matching_namsespace, metric_id)
229+
rows.append(runtime_metric.as_dict())
230+
if len(not_matching_namespace) > 0:
231+
rows += get_generic_distributions(not_matching_namespace, metric_id)
231232
return rows
232233

233234

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,16 @@
1+
#
2+
# Licensed to the Apache Software Foundation (ASF) under one or more
3+
# contributor license agreements. See the NOTICE file distributed with
4+
# this work for additional information regarding copyright ownership.
5+
# The ASF licenses this file to You under the Apache License, Version 2.0
6+
# (the "License"); you may not use this file except in compliance with
7+
# the License. You may obtain a copy of the License at
8+
#
9+
# http://www.apache.org/licenses/LICENSE-2.0
10+
#
11+
# Unless required by applicable law or agreed to in writing, software
12+
# distributed under the License is distributed on an "AS IS" BASIS,
13+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
# See the License for the specific language governing permissions and
15+
# limitations under the License.
16+
#

0 commit comments

Comments
 (0)