[
https://issues.apache.org/jira/browse/BEAM-6522?focusedWorklogId=231877&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-231877
]
ASF GitHub Bot logged work on BEAM-6522:
----------------------------------------
Author: ASF GitHub Bot
Created on: 24/Apr/19 06:15
Start Date: 24/Apr/19 06:15
Worklog Time Spent: 10m
Work Description: tvalentyn commented on pull request #8130: [BEAM-6522]
Fix fastavro on Python 3
URL: https://github.com/apache/beam/pull/8130#discussion_r277967068
##########
File path: sdks/python/apache_beam/io/avroio_test.py
##########
@@ -472,10 +476,55 @@ def test_sink_transform_snappy(self):
assert_that(readback, equal_to([json.dumps(r) for r in self.RECORDS]))
-class TestFastAvro(TestAvro):
[email protected](sys.version_info[0] == 3 and
+ os.environ.get('RUN_SKIPPED_PY3_TESTS') != '1',
+ 'This test still needs to be fixed on Python 3. '
+ 'TODO: BEAM-6522.')
+class TestAvro(AvroBase, unittest.TestCase):
+ def __init__(self, methodName='runTest'):
+ super(TestAvro, self).__init__(methodName)
+ self.use_fastavro = False
+ self.SCHEMA = Parse(self.SCHEMA_STRING)
+
+ def _write_data(self,
+ directory=None,
+ prefix=tempfile.template,
+ codec='null',
+ count=len(RECORDS)):
+ with tempfile.NamedTemporaryFile(delete=False,
+ dir=directory,
+ prefix=prefix) as f:
+ writer = DataFileWriter(f, DatumWriter(), self.SCHEMA, codec=codec)
+ len_records = len(self.RECORDS)
+ for i in range(count):
+ writer.append(self.RECORDS[i % len_records])
+ writer.close()
+ self._temp_files.append(f.name)
+ return f.name
+
+
+class TestFastAvro(AvroBase, unittest.TestCase):
def __init__(self, methodName='runTest'):
super(TestFastAvro, self).__init__(methodName)
self.use_fastavro = True
+ self.SCHEMA = parse_schema(json.loads(self.SCHEMA_STRING))
+
+ def _write_data(self,
+ directory=None,
+ prefix=tempfile.template,
+ codec='null',
+ count=len(RECORDS),
+ sync_interval=16000):
Review comment:
Let's not redefine default values for `writer()` method in the test.
Currently the defaults for `codec` and `sync_interval` match the respective
defaults in `fastavro/avro`, but if the library implementation changes the
default in the future, the test may be still hardcoding old value. How about
something like:
```
def _write_data(self,
directory=None,
prefix=tempfile.template,
count=len(RECORDS),
**kwargs):
...
writer(f, self.SCHEMA, all_records, **kwargs)
```
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 231877)
Time Spent: 4h 50m (was: 4h 40m)
> Avro RecordSchema class is not picklable
> ----------------------------------------
>
> Key: BEAM-6522
> URL: https://issues.apache.org/jira/browse/BEAM-6522
> Project: Beam
> Issue Type: Sub-task
> Components: sdk-py-core
> Reporter: Robbe
> Assignee: Robbe
> Priority: Major
> Labels: triaged
> Time Spent: 4h 50m
> Remaining Estimate: 0h
>
> The avroio module still has 4 failing tests. This is actually 2 times the
> same 2 tests, both for Avro and Fastavro.
> *apache_beam.io.avroio_test.TestAvro.test_sink_transform*
> *apache_beam.io.avroio_test.TestFastAvro.test_sink_transform*
> fail with:
> {code:java}
> Traceback (most recent call last):
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/io/avroio_test.py",
> line 432, in test_sink_transform
> | avroio.WriteToAvro(path, self.SCHEMA, use_fastavro=self.use_fastavro)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/pvalue.py", line
> 112, in __or__
> return self.pipeline.apply(ptransform, self)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/pipeline.py", line
> 515, in apply
> pvalueish_result = self.runner.apply(transform, pvalueish, self._options)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/runners/runner.py",
> line 193, in apply
> return m(transform, input, options)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/runners/runner.py",
> line 199, in apply_PTransform
> return transform.expand(input)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/io/avroio.py", line
> 528, in expand
> return pcoll | beam.io.iobase.Write(self._sink)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/pvalue.py", line
> 112, in __or__
> return self.pipeline.apply(ptransform, self)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/pipeline.py", line
> 515, in apply
> pvalueish_result = self.runner.apply(transform, pvalueish, self._options)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/runners/runner.py",
> line 193, in apply
> return m(transform, input, options)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/runners/runner.py",
> line 199, in apply_PTransform
> return transform.expand(input)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/io/iobase.py", line
> 960, in expand
> return pcoll | WriteImpl(self.sink)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/pvalue.py", line
> 112, in __or__
> return self.pipeline.apply(ptransform, self)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/pipeline.py", line
> 515, in apply
> pvalueish_result = self.runner.apply(transform, pvalueish, self._options)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/runners/runner.py",
> line 193, in apply
> return m(transform, input, options)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/runners/runner.py",
> line 199, in apply_PTransform
> return transform.expand(input)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/io/iobase.py", line
> 979, in expand
> lambda _, sink: sink.initialize_write(), self.sink)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/transforms/core.py",
> line 1103, in Map
> pardo = FlatMap(wrapper, *args, **kwargs)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/transforms/core.py",
> line 1054, in FlatMap
> pardo = ParDo(CallableWrapperDoFn(fn), *args, **kwargs)
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/transforms/core.py",
> line 864, in __init__
> super(ParDo, self).__init__(fn, *args, **kwargs)
> File
> "/home/robbe/workspace/beam/sdks/python/apache_beam/transforms/ptransform.py",
> line 646, in __init__
> self.args = pickler.loads(pickler.dumps(self.args))
> File
> "/home/robbe/workspace/beam/sdks/python/apache_beam/internal/pickler.py",
> line 247, in loads
> return dill.loads(s)
> File
> "/home/robbe/workspace/beam/sdks/python/.eggs/dill-0.2.9-py3.5.egg/dill/_dill.py",
> line 317, in loads
> return load(file, ignore)
> File
> "/home/robbe/workspace/beam/sdks/python/.eggs/dill-0.2.9-py3.5.egg/dill/_dill.py",
> line 305, in load
> obj = pik.load()
> File
> "/home/robbe/workspace/beam/sdks/python/target/.tox/py3/lib/python3.5/site-packages/avro/schema.py",
> line 173, in __setitem__
> % (key, value, self))
> Exception: Attempting to map key 'favorite_color' to value <avro.schema.Field
> object at 0x7f8f72d0d0b8> in ImmutableDict {}
> {code}
>
> *apache_beam.io.avroio_test.TestAvro.test_split_points*
> *apache_beam.io.avroio_test.TestFastAvro.test_split_points*
> fail with:
>
> {code:java}
> Traceback (most recent call last):
> File "/home/robbe/workspace/beam/sdks/python/apache_beam/io/avroio_test.py",
> line 308, in test_split_points
> self.assertEquals(split_points_report[-10:], [(2, 1)] * 10)
> AssertionError: Lists differ: [(10, 1), (10, 1), (10, 1), (10, 1), (10, 1[42
> chars], 1)] != [(2, 1), (2, 1), (2, 1), (2, 1), (2, 1), (2[32 chars], 1)]
> First differing element 0:
> (10, 1)
> (2, 1)
> + [(2, 1), (2, 1), (2, 1), (2, 1), (2, 1), (2, 1), (2, 1), (2, 1), (2, 1),
> (2, 1)]
> - [(10, 1),
> - (10, 1),
> - (10, 1),
> - (10, 1),
> - (10, 1),
> - (10, 1),
> - (10, 1),
> - (10, 1),
> - (10, 1),
> - (10, 1)]
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)