-
Notifications
You must be signed in to change notification settings - Fork 4.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[BEAM-13016] Remove avro-python3 dependency from Beam #15900
Changes from all commits
8d223d8
99f837b
47192b5
3f9bc65
cb6e47c
ee27084
bcb3d39
f178f32
c034fde
054594d
0e9e531
5c1eb58
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,9 +17,11 @@ | |
|
||
"""End-to-end test for Avro IO's fastavro support. | ||
|
||
Writes a configurable number of records to a temporary location with each of | ||
{avro,fastavro}, then reads them back in, joins the two read datasets, and | ||
verifies they have the same elements. | ||
Writes a configurable number of records to a temporary location with fastavro, | ||
then reads them back in from source, joins the generated records and records | ||
that are read from the source, and verifies they have the same elements. | ||
|
||
|
||
|
||
Usage: | ||
|
||
|
@@ -51,7 +53,6 @@ | |
import uuid | ||
|
||
import pytest | ||
from avro.schema import Parse | ||
from fastavro import parse_schema | ||
|
||
from apache_beam.io.avroio import ReadAllFromAvro | ||
|
@@ -78,6 +79,18 @@ def record(i): | |
} | ||
|
||
|
||
def assertEqual(l, r): | ||
if l != r: | ||
raise BeamAssertException('Assertion failed: %s == %s' % (l, r)) | ||
|
||
|
||
def check(element): | ||
assert element['color'] in COLORS | ||
assert element['label'] in LABELS | ||
assertEqual( | ||
sorted(element.keys()), ['color', 'label', 'number', 'number_str']) | ||
|
||
|
||
class FastavroIT(unittest.TestCase): | ||
|
||
SCHEMA_STRING = ''' | ||
|
@@ -102,6 +115,7 @@ def setUp(self): | |
def test_avro_it(self): | ||
num_records = self.test_pipeline.get_option('records') | ||
num_records = int(num_records) if num_records else 1000000 | ||
fastavro_output = '/'.join([self.output, 'fastavro']) | ||
|
||
# Seed a `PCollection` with indices that will each be FlatMap'd into | ||
# `batch_size` records, to avoid having a too-large list in memory at | ||
|
@@ -123,65 +137,44 @@ def batch_indices(start): | |
| 'expand-batches' >> FlatMap(batch_indices) \ | ||
| 'create-records' >> Map(record) | ||
|
||
fastavro_output = '/'.join([self.output, 'fastavro']) | ||
avro_output = '/'.join([self.output, 'avro']) | ||
|
||
# pylint: disable=expression-not-assigned | ||
records_pcoll \ | ||
| 'write_fastavro' >> WriteToAvro( | ||
fastavro_output, | ||
parse_schema(json.loads(self.SCHEMA_STRING)), | ||
use_fastavro=True | ||
) | ||
result = self.test_pipeline.run() | ||
result.wait_until_finish() | ||
fastavro_pcoll = self.test_pipeline \ | ||
| 'create-fastavro' >> Create(['%s*' % fastavro_output]) \ | ||
| 'read-fastavro' >> ReadAllFromAvro() | ||
|
||
# pylint: disable=expression-not-assigned | ||
records_pcoll \ | ||
| 'write_avro' >> WriteToAvro( | ||
avro_output, | ||
Parse(self.SCHEMA_STRING), | ||
use_fastavro=False | ||
) | ||
mapped_fastavro_pcoll = fastavro_pcoll | "map_fastavro" >> Map( | ||
lambda x: (x['number'], x)) | ||
mapped_record_pcoll = records_pcoll | "map_record" >> Map( | ||
lambda x: (x['number'], x)) | ||
|
||
def validate_record(elem): | ||
v = elem[1] | ||
|
||
def assertEqual(l, r): | ||
if l != r: | ||
raise BeamAssertException('Assertion failed: %s == %s' % (l, r)) | ||
|
||
assertEqual(sorted(v.keys()), ['fastavro', 'record_pcoll']) | ||
record_pcoll_values = v['record_pcoll'] | ||
fastavro_values = v['fastavro'] | ||
assertEqual(record_pcoll_values, fastavro_values) | ||
assertEqual(len(record_pcoll_values), 1) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What guarantees that we have unique keys for each record (so that after GBK we only have 1 value per key)? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We will have unique keys for each record as the keys are integers in ascending order. This is an example of the output after CoGroupByKey. Here the number of records are 20 and keys are the integer(number attribute of the pcollection)
|
||
|
||
{ | ||
"record_pcoll": mapped_record_pcoll, "fastavro": mapped_fastavro_pcoll | ||
} | CoGroupByKey() | Map(validate_record) | ||
|
||
result = self.test_pipeline.run() | ||
result.wait_until_finish() | ||
assert result.state == PipelineState.DONE | ||
|
||
with TestPipeline(is_integration_test=True) as fastavro_read_pipeline: | ||
|
||
fastavro_records = \ | ||
fastavro_read_pipeline \ | ||
| 'create-fastavro' >> Create(['%s*' % fastavro_output]) \ | ||
| 'read-fastavro' >> ReadAllFromAvro(use_fastavro=True) \ | ||
| Map(lambda rec: (rec['number'], rec)) | ||
|
||
avro_records = \ | ||
fastavro_read_pipeline \ | ||
| 'create-avro' >> Create(['%s*' % avro_output]) \ | ||
| 'read-avro' >> ReadAllFromAvro(use_fastavro=False) \ | ||
| Map(lambda rec: (rec['number'], rec)) | ||
|
||
def check(elem): | ||
v = elem[1] | ||
|
||
def assertEqual(l, r): | ||
if l != r: | ||
raise BeamAssertException('Assertion failed: %s == %s' % (l, r)) | ||
|
||
assertEqual(sorted(v.keys()), ['avro', 'fastavro']) | ||
avro_values = v['avro'] | ||
fastavro_values = v['fastavro'] | ||
assertEqual(avro_values, fastavro_values) | ||
assertEqual(len(avro_values), 1) | ||
|
||
# pylint: disable=expression-not-assigned | ||
{ | ||
'avro': avro_records, | ||
'fastavro': fastavro_records | ||
} \ | ||
| CoGroupByKey() \ | ||
| Map(check) | ||
|
||
self.addCleanup(delete_files, [self.output]) | ||
self.addCleanup(delete_files, [self.output]) | ||
assert result.state == PipelineState.DONE | ||
|
||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For future changes, prefer not to use \ for concatenating strings, since it's error prone. Using brackets is preferable.
From PEP8: (https://www.python.org/dev/peps/pep-0008/)
(this was a preexisting issue in this file).